You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/11/22 02:39:03 UTC

svn commit: r1544389 [2/2] - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/server/name...

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java?rev=1544389&r1=1544388&r2=1544389&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java Fri Nov 22 01:39:02 2013
@@ -43,8 +43,7 @@ public class TestCommitBlockSynchronizat
   private static final long length = 200;
   private static final long genStamp = 300;
 
-  private FSNamesystem makeNameSystemSpy(Block block,
-                                         INodeFileUnderConstruction file)
+  private FSNamesystem makeNameSystemSpy(Block block, INodeFile file)
       throws IOException {
     Configuration conf = new Configuration();
     FSImage image = new FSImage(conf);
@@ -58,21 +57,26 @@ public class TestCommitBlockSynchronizat
     blockInfo.setGenerationStamp(genStamp);
     blockInfo.initializeBlockRecovery(genStamp);
     doReturn(true).when(file).removeLastBlock(any(Block.class));
+    doReturn(true).when(file).isUnderConstruction();
 
     doReturn(blockInfo).when(namesystemSpy).getStoredBlock(any(Block.class));
     doReturn("").when(namesystemSpy).closeFileCommitBlocks(
-        any(INodeFileUnderConstruction.class),
-        any(BlockInfo.class));
+        any(INodeFile.class), any(BlockInfo.class));
     doReturn("").when(namesystemSpy).persistBlocks(
-        any(INodeFileUnderConstruction.class), anyBoolean());
+        any(INodeFile.class), anyBoolean());
     doReturn(mock(FSEditLog.class)).when(namesystemSpy).getEditLog();
 
     return namesystemSpy;
   }
 
+  private INodeFile mockFileUnderConstruction() {
+    INodeFile file = mock(INodeFile.class);
+    return file;
+  }
+
   @Test
   public void testCommitBlockSynchronization() throws IOException {
-    INodeFileUnderConstruction file = mock(INodeFileUnderConstruction.class);
+    INodeFile file = mockFileUnderConstruction();
     Block block = new Block(blockId, length, genStamp);
     FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
     DatanodeID[] newTargets = new DatanodeID[0];
@@ -100,7 +104,7 @@ public class TestCommitBlockSynchronizat
 
   @Test
   public void testCommitBlockSynchronization2() throws IOException {
-    INodeFileUnderConstruction file = mock(INodeFileUnderConstruction.class);
+    INodeFile file = mockFileUnderConstruction();
     Block block = new Block(blockId, length, genStamp);
     FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
     DatanodeID[] newTargets = new DatanodeID[0];
@@ -124,7 +128,7 @@ public class TestCommitBlockSynchronizat
 
   @Test
   public void testCommitBlockSynchronizationWithDelete() throws IOException {
-    INodeFileUnderConstruction file = mock(INodeFileUnderConstruction.class);
+    INodeFile file = mockFileUnderConstruction();
     Block block = new Block(blockId, length, genStamp);
     FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
     DatanodeID[] newTargets = new DatanodeID[0];
@@ -144,7 +148,7 @@ public class TestCommitBlockSynchronizat
 
   @Test
   public void testCommitBlockSynchronizationWithClose() throws IOException {
-    INodeFileUnderConstruction file = mock(INodeFileUnderConstruction.class);
+    INodeFile file = mockFileUnderConstruction();
     Block block = new Block(blockId, length, genStamp);
     FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
     DatanodeID[] newTargets = new DatanodeID[0];
@@ -171,7 +175,7 @@ public class TestCommitBlockSynchronizat
   @Test
   public void testCommitBlockSynchronizationWithCloseAndNonExistantTarget()
       throws IOException {
-    INodeFileUnderConstruction file = mock(INodeFileUnderConstruction.class);
+    INodeFile file = mockFileUnderConstruction();
     Block block = new Block(blockId, length, genStamp);
     FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
     DatanodeID[] newTargets = new DatanodeID[]{

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java?rev=1544389&r1=1544388&r2=1544389&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java Fri Nov 22 01:39:02 2013
@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.HdfsConfig
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@@ -152,9 +153,10 @@ public class TestEditLog {
       FSEditLog editLog = namesystem.getEditLog();
 
       for (int i = 0; i < numTransactions; i++) {
-        INodeFileUnderConstruction inode = new INodeFileUnderConstruction(
-            namesystem.allocateNewInodeId(), p, replication, blockSize, 0, "",
-            "", null);
+        INodeFile inode = new INodeFile(namesystem.allocateNewInodeId(), null,
+            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
+        inode.toUnderConstruction("", "", null);
+
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false);
         editLog.logCloseFile("/filename" + (startIndex + i), inode);
         editLog.logSync();

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java?rev=1544389&r1=1544388&r2=1544389&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java Fri Nov 22 01:39:02 2013
@@ -29,6 +29,8 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 
+import junit.framework.Assert;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -59,6 +61,7 @@ import org.apache.hadoop.hdfs.protocol.L
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -282,14 +285,6 @@ public class TestINodeFile {
         assertTrue(fnfe.getMessage().contains("File does not exist"));
       }
 
-      //cast to INodeFileUnderConstruction, should fail
-      try {
-        INodeFileUnderConstruction.valueOf(from, path);
-        fail();
-      } catch(FileNotFoundException fnfe) {
-        assertTrue(fnfe.getMessage().contains("File does not exist"));
-      }
-
       //cast to INodeDirectory, should fail
       try {
         INodeDirectory.valueOf(from, path);
@@ -306,14 +301,6 @@ public class TestINodeFile {
       final INodeFile f = INodeFile.valueOf(from, path);
       assertTrue(f == from);
 
-      //cast to INodeFileUnderConstruction, should fail
-      try {
-        INodeFileUnderConstruction.valueOf(from, path);
-        fail();
-      } catch(IOException ioe) {
-        assertTrue(ioe.getMessage().contains("File is not under construction"));
-      }
-
       //cast to INodeDirectory, should fail
       try {
         INodeDirectory.valueOf(from, path);
@@ -324,19 +311,14 @@ public class TestINodeFile {
     }
 
     {//cast from INodeFileUnderConstruction
-      final INode from = new INodeFileUnderConstruction(
-          INodeId.GRANDFATHER_INODE_ID, perm, replication, 0L, 0L, "client",
-          "machine", null);
+      final INode from = new INodeFile(
+          INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication, 1024L);
+      from.asFile().toUnderConstruction("client", "machine", null);
     
       //cast to INodeFile, should success
       final INodeFile f = INodeFile.valueOf(from, path);
       assertTrue(f == from);
 
-      //cast to INodeFileUnderConstruction, should success
-      final INodeFileUnderConstruction u = INodeFileUnderConstruction.valueOf(
-          from, path);
-      assertTrue(u == from);
-
       //cast to INodeDirectory, should fail
       try {
         INodeDirectory.valueOf(from, path);
@@ -358,14 +340,6 @@ public class TestINodeFile {
         assertTrue(fnfe.getMessage().contains("Path is not a file"));
       }
 
-      //cast to INodeFileUnderConstruction, should fail
-      try {
-        INodeFileUnderConstruction.valueOf(from, path);
-        fail();
-      } catch(FileNotFoundException fnfe) {
-        assertTrue(fnfe.getMessage().contains("Path is not a file"));
-      }
-
       //cast to INodeDirectory, should success
       final INodeDirectory d = INodeDirectory.valueOf(from, path);
       assertTrue(d == from);
@@ -1015,4 +989,24 @@ public class TestINodeFile {
       }
     }
   }
+
+  @Test
+  public void testFileUnderConstruction() {
+    replication = 3;
+    final INodeFile file = new INodeFile(INodeId.GRANDFATHER_INODE_ID, null,
+        perm, 0L, 0L, null, replication, 1024L);
+    assertFalse(file.isUnderConstruction());
+
+    final String clientName = "client";
+    final String clientMachine = "machine";
+    file.toUnderConstruction(clientName, clientMachine, null);
+    assertTrue(file.isUnderConstruction());
+    FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
+    assertEquals(clientName, uc.getClientName());
+    assertEquals(clientMachine, uc.getClientMachine());
+    Assert.assertNull(uc.getClientNode());
+
+    file.toCompleteFile(Time.now());
+    assertFalse(file.isUnderConstruction());
+  }
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java?rev=1544389&r1=1544388&r2=1544389&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java Fri Nov 22 01:39:02 2013
@@ -66,7 +66,6 @@ import org.apache.hadoop.hdfs.protocol.C
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
@@ -719,8 +718,8 @@ public class TestRetryCacheWithHA {
 
     @Override
     boolean checkNamenodeBeforeReturn() throws Exception {
-      INodeFileUnderConstruction fileNode = (INodeFileUnderConstruction) cluster
-          .getNamesystem(0).getFSDirectory().getINode4Write(file).asFile();
+      INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
+          .getINode4Write(file).asFile();
       BlockInfoUnderConstruction blkUC = 
           (BlockInfoUnderConstruction) (fileNode.getBlocks())[1];
       int datanodeNum = blkUC.getExpectedLocations().length;

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java?rev=1544389&r1=1544388&r2=1544389&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java Fri Nov 22 01:39:02 2013
@@ -1227,8 +1227,9 @@ public class TestRenameWithSnapshots {
       out.write(content);
       fooRef = fsdir.getINode4Write(foo2.toString());
       assertTrue(fooRef instanceof INodeReference.DstReference);
-      INode fooNode = fooRef.asFile();
-      assertTrue(fooNode instanceof INodeFileUnderConstructionWithSnapshot);
+      INodeFile fooNode = fooRef.asFile();
+      assertTrue(fooNode instanceof INodeFileWithSnapshot);
+      assertTrue(fooNode.isUnderConstruction());
     } finally {
       if (out != null) {
         out.close();
@@ -1237,8 +1238,9 @@ public class TestRenameWithSnapshots {
     
     fooRef = fsdir.getINode4Write(foo2.toString());
     assertTrue(fooRef instanceof INodeReference.DstReference);
-    INode fooNode = fooRef.asFile();
+    INodeFile fooNode = fooRef.asFile();
     assertTrue(fooNode instanceof INodeFileWithSnapshot);
+    assertFalse(fooNode.isUnderConstruction());
     
     restartClusterAndCheckImage(true);
   }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java?rev=1544389&r1=1544388&r2=1544389&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java Fri Nov 22 01:39:02 2013
@@ -314,7 +314,9 @@ public class TestSnapshotBlocksMap {
     assertEquals(BLOCKSIZE, blks[0].getNumBytes());
   }
 
-  /** Make sure we delete 0-sized block when deleting an INodeFileUC */
+  /**
+   * Make sure we delete 0-sized block when deleting an under-construction file
+   */
   @Test
   public void testDeletionWithZeroSizeBlock2() throws Exception {
     final Path foo = new Path("/foo");