Browse Source

HDFS-9690. ClientProtocol.addBlock is not idempotent after HDFS-8071.(Contributed by Tsz Wo Nicholas Sze)

Vinayakumar B 9 years ago
parent
commit
07704aab84

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -68,6 +68,9 @@ Release 2.7.3 - UNRELEASED
     HDFS-9625. set replication for empty file failed when set storage policy
     (DENG FEI via vinayakumarb)
 
+    HDFS-9690. ClientProtocol.addBlock is not idempotent after HDFS-8071.
+    (szetszwo)
+
 Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -3070,18 +3070,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       src = dir.resolvePath(pc, src, pathComponents);
       FileState fileState = analyzeFileState(
           src, fileId, clientName, previous, onRetryBlock);
+      if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
+        // This is a retry. No need to generate new locations.
+        // Use the last block if it has locations.
+        return null;
+      }
+
       final INodeFile pendingFile = fileState.inode;
-      // Check if the penultimate block is minimally replicated
       if (!checkFileProgress(src, pendingFile, false)) {
         throw new NotReplicatedYetException("Not replicated yet: " + src);
       }
       src = fileState.path;
 
-      if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
-        // This is a retry. No need to generate new locations.
-        // Use the last block if it has locations.
-        return null;
-      }
       if (pendingFile.getBlocks().length >= maxBlocksPerFile) {
         throw new IOException("File has reached the limit on maximum number of"
             + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY

+ 27 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -434,19 +434,37 @@ public class TestDFSClientRetries {
       // Make the call to addBlock() get called twice, as if it were retried
       // due to an IPC issue.
       doAnswer(new Answer<LocatedBlock>() {
-        @Override
-        public LocatedBlock answer(InvocationOnMock invocation) throws Throwable {
-          LocatedBlock ret = (LocatedBlock) invocation.callRealMethod();
+        private int getBlockCount(LocatedBlock ret) throws IOException {
           LocatedBlocks lb = cluster.getNameNodeRpc().getBlockLocations(src, 0, Long.MAX_VALUE);
-          int blockCount = lb.getLocatedBlocks().size();
           assertEquals(lb.getLastLocatedBlock().getBlock(), ret.getBlock());
-          
+          return lb.getLocatedBlocks().size();
+        }
+
+        @Override
+        public LocatedBlock answer(InvocationOnMock invocation) throws Throwable {
+          LOG.info("Called addBlock: "
+              + Arrays.toString(invocation.getArguments()));
+
+          // call first time
+          // warp NotReplicatedYetException with RemoteException as rpc does.
+          final LocatedBlock ret;
+          try {
+            ret = (LocatedBlock) invocation.callRealMethod();
+          } catch(NotReplicatedYetException e) {
+            throw new RemoteException(e.getClass().getName(), e.getMessage());
+          }
+          final int blockCount = getBlockCount(ret);
+
           // Retrying should result in a new block at the end of the file.
           // (abandoning the old one)
-          LocatedBlock ret2 = (LocatedBlock) invocation.callRealMethod();
-          lb = cluster.getNameNodeRpc().getBlockLocations(src, 0, Long.MAX_VALUE);
-          int blockCount2 = lb.getLocatedBlocks().size();
-          assertEquals(lb.getLastLocatedBlock().getBlock(), ret2.getBlock());
+          // It should not have NotReplicatedYetException.
+          final LocatedBlock ret2;
+          try {
+            ret2 = (LocatedBlock) invocation.callRealMethod();
+          } catch(NotReplicatedYetException e) {
+            throw new AssertionError("Unexpected exception", e);
+          }
+          final int blockCount2 = getBlockCount(ret2);
 
           // We shouldn't have gained an extra block by the RPC.
           assertEquals(blockCount, blockCount2);