瀏覽代碼

HDFS-3385. The last block of INodeFileUnderConstruction is not necessarily a BlockInfoUnderConstruction, so do not cast it in FSNamesystem.recoverLeaseInternal(..).

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1336976 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 年之前
父節點
當前提交
ba6360e493

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

@@ -644,6 +644,10 @@ Release 2.0.0 - UNRELEASED
     HDFS-3395. NN doesn't start with HA+security enabled and HTTP address
     set to 0.0.0.0. (atm)
 
+    HDFS-3385. The last block of INodeFileUnderConstruction is not
+    necessarily a BlockInfoUnderConstruction, so do not cast it in
+    FSNamesystem.recoverLeaseInternal(..).  (szetszwo)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

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

@@ -1783,24 +1783,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                 "Failed to close file " + src +
                 ". Lease recovery is in progress. Try again later.");
         } else {
-          BlockInfoUnderConstruction lastBlock=pendingFile.getLastBlock();
-          if(lastBlock != null && lastBlock.getBlockUCState() ==
-            BlockUCState.UNDER_RECOVERY) {
-            throw new RecoveryInProgressException(
-              "Recovery in progress, file [" + src + "], " +
-              "lease owner [" + lease.getHolder() + "]");
-            } else {
-              throw new AlreadyBeingCreatedException(
-                "Failed to create file [" + src + "] for [" + holder +
-                "] on client [" + clientMachine +
-                "], because this file is already being created by [" +
-                pendingFile.getClientName() + "] on [" +
-                pendingFile.getClientMachine() + "]");
-            }
-         }
+          final BlockInfo lastBlock = pendingFile.getLastBlock();
+          if (lastBlock != null
+              && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
+            throw new RecoveryInProgressException("Recovery in progress, file ["
+                + src + "], " + "lease owner [" + lease.getHolder() + "]");
+          } else {
+            throw new AlreadyBeingCreatedException("Failed to create file ["
+                + src + "] for [" + holder + "] on client [" + clientMachine
+                + "], because this file is already being created by ["
+                + pendingFile.getClientName() + "] on ["
+                + pendingFile.getClientMachine() + "]");
+          }
+        }
       }
     }
-
   }
 
   /**

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -31,12 +31,15 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.ipc.RemoteException;
+import org.junit.Assert;
 import org.junit.Test;
 
 /**
@@ -295,4 +298,43 @@ public class TestFileAppend{
       cluster.shutdown();
     }
   }
+
+  /** Test two consecutive appends on a file with a full block. */
+  @Test
+  public void testAppendTwice() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    final FileSystem fs1 = cluster.getFileSystem();
+    final FileSystem fs2 = AppendTestUtil.createHdfsWithDifferentUsername(conf);
+    try {
+  
+      final Path p = new Path("/testAppendTwice/foo");
+      final int len = 1 << 16;
+      final byte[] fileContents = AppendTestUtil.initBuffer(len);
+
+      {
+        // create a new file with a full block.
+        FSDataOutputStream out = fs2.create(p, true, 4096, (short)1, len);
+        out.write(fileContents, 0, len);
+        out.close();
+      }
+  
+      //1st append does not add any data so that the last block remains full
+      //and the last block in INodeFileUnderConstruction is a BlockInfo
+      //but not BlockInfoUnderConstruction. 
+      fs2.append(p);
+      
+      //2nd append should get AlreadyBeingCreatedException
+      fs1.append(p);
+      Assert.fail();
+    } catch(RemoteException re) {
+      AppendTestUtil.LOG.info("Got an exception:", re);
+      Assert.assertEquals(AlreadyBeingCreatedException.class.getName(),
+          re.getClassName());
+    } finally {
+      fs2.close();
+      fs1.close();
+      cluster.shutdown();
+    }
+  }
 }