From cb282d5b89fdece4719cc4ad37a6e27f13371534 Mon Sep 17 00:00:00 2001 From: cnauroth Date: Thu, 22 Oct 2015 14:29:57 -0700 Subject: [PATCH] HADOOP-12484. Single File Rename Throws Incorrectly In Potential Race Condition Scenarios. Contributed by Gaurav Kanade. --- .../hadoop-common/CHANGES.txt | 3 ++ .../fs/azure/NativeAzureFileSystem.java | 32 ++++++++++++++++--- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 74c62cb0d94..87ba2ba69ce 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -1338,6 +1338,9 @@ Release 2.8.0 - UNRELEASED HADOOP-12334. Change Mode Of Copy Operation of HBase WAL Archiving to bypass Azure Storage Throttling after retries. (Gaurav Kanade via cnauroth) + HADOOP-12484. Single File Rename Throws Incorrectly In Potential Race + Condition Scenarios. (Gaurav Kanade via cnauroth) + Release 2.7.2 - UNRELEASED INCOMPATIBLE CHANGES diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java index b963d5a18ee..9305b24e851 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java @@ -545,10 +545,32 @@ public class NativeAzureFileSystem extends FileSystem { // Get a lease on source to block write access. String srcName = fs.pathToKey(srcFile); - SelfRenewingLease lease = fs.acquireLease(srcFile); - - // Delete the file. This will free the lease too. - fs.getStoreInterface().delete(srcName, lease); + SelfRenewingLease lease = null; + try { + lease = fs.acquireLease(srcFile); + // Delete the file. This will free the lease too. + fs.getStoreInterface().delete(srcName, lease); + } catch(AzureException e) { + String errorCode = ""; + try { + StorageException e2 = (StorageException) e.getCause(); + errorCode = e2.getErrorCode(); + } catch(Exception e3) { + // do nothing if cast fails + } + // If the rename already finished do nothing + if(!errorCode.equals("BlobNotFound")){ + throw e; + } + } finally { + try { + if(lease != null){ + lease.free(); + } + } catch(StorageException e) { + LOG.warn("Unable to free lease because: " + e.getMessage()); + } + } } else if (!srcExists && dstExists) { // The rename already finished, so do nothing. @@ -2442,4 +2464,4 @@ public class NativeAzureFileSystem extends FileSystem { } } } -} \ No newline at end of file +}