Browse Source

HDFS-9252. Change TestFileTruncate to use FsDatasetTestUtils to get block file size and genstamp. (Lei (Eddy) Xu via cmccabe)

(cherry picked from commit dfbde3fc511495ac998f07d674a87355de75fc04)
Colin Patrick Mccabe 9 years ago
parent
commit
0f9f9ba3cb

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

@@ -795,6 +795,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9369. Use ctest to run tests for hadoop-hdfs-native-client. (wheat9)
 
+    HDFS-9252. Change TestFileTruncate to use FsDatasetTestUtils to get block
+    file size and genstamp. (Lei (Eddy) Xu via cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java

@@ -75,14 +75,15 @@ public class FsDatasetUtil {
    * Find the meta-file for the specified block file
    * and then return the generation stamp from the name of the meta-file.
    */
-  static long getGenerationStampFromFile(File[] listdir, File blockFile) {
+  static long getGenerationStampFromFile(File[] listdir, File blockFile)
+      throws IOException {
     String blockName = blockFile.getName();
     for (int j = 0; j < listdir.length; j++) {
       String path = listdir[j].getName();
       if (!path.startsWith(blockName)) {
         continue;
       }
-      if (blockFile == listdir[j]) {
+      if (blockFile.getCanonicalPath().equals(listdir[j].getCanonicalPath())) {
         continue;
       }
       return Block.getGenerationStamp(listdir[j].getName());

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/FsDatasetTestUtils.java

@@ -232,4 +232,14 @@ public interface FsDatasetTestUtils {
    * Obtain the raw capacity of underlying storage per DataNode.
    */
   long getRawCapacity() throws IOException;
+
+  /**
+   * Get the persistently stored length of the block.
+   */
+  long getStoredDataLength(ExtendedBlock block) throws IOException;
+
+  /**
+   * Get the persistently stored generation stamp.
+   */
+  long getStoredGenerationStamp(ExtendedBlock block) throws IOException;
 }

+ 21 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
@@ -175,6 +176,10 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
     dataset = (FsDatasetImpl) datanode.getFSDataset();
   }
 
+  private File getBlockFile(ExtendedBlock eb) throws IOException {
+    return dataset.getBlockFile(eb.getBlockPoolId(), eb.getBlockId());
+  }
+
   /**
    * Return a materialized replica from the FsDatasetImpl.
    */
@@ -235,7 +240,6 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
     return rip;
   }
 
-
   @Override
   public Replica createRBW(ExtendedBlock eb) throws IOException {
     try (FsVolumeReferences volumes = dataset.getFsVolumeReferences()) {
@@ -343,4 +347,20 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
       return df.getCapacity();
     }
   }
+
+  @Override
+  public long getStoredDataLength(ExtendedBlock block) throws IOException {
+    File f = getBlockFile(block);
+    try (RandomAccessFile raf = new RandomAccessFile(f, "r")) {
+      return raf.length();
+    }
+  }
+
+  @Override
+  public long getStoredGenerationStamp(ExtendedBlock block) throws IOException {
+    File f = getBlockFile(block);
+    File dir = f.getParentFile();
+    File[] files = FileUtil.listFiles(dir);
+    return FsDatasetUtil.getGenerationStampFromFile(files, f);
+  }
 }

+ 18 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
 import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -698,11 +699,11 @@ public class TestFileTruncate {
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // Old replica is disregarded and replaced with the truncated one
-    assertEquals(cluster.getBlockFile(dn, newBlock.getBlock()).length(), 
+    FsDatasetTestUtils utils = cluster.getFsDatasetTestUtils(dn);
+    assertEquals(utils.getStoredDataLength(newBlock.getBlock()),
         newBlock.getBlockSize());
-    assertTrue(cluster.getBlockMetadataFile(dn, 
-        newBlock.getBlock()).getName().endsWith(
-            newBlock.getBlock().getGenerationStamp() + ".meta"));
+    assertEquals(utils.getStoredGenerationStamp(newBlock.getBlock()),
+        newBlock.getBlock().getGenerationStamp());
 
     // Validate the file
     FileStatus fileStatus = fs.getFileStatus(p);
@@ -752,15 +753,15 @@ public class TestFileTruncate {
 
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
+    FsDatasetTestUtils utils = cluster.getFsDatasetTestUtils(dn);
     // New block is replicated to dn1
-    assertEquals(cluster.getBlockFile(dn, newBlock.getBlock()).length(), 
+    assertEquals(utils.getStoredDataLength(newBlock.getBlock()),
         newBlock.getBlockSize());
     // Old replica exists too since there is snapshot
-    assertEquals(cluster.getBlockFile(dn, oldBlock.getBlock()).length(), 
+    assertEquals(utils.getStoredDataLength(oldBlock.getBlock()),
         oldBlock.getBlockSize());
-    assertTrue(cluster.getBlockMetadataFile(dn, 
-        oldBlock.getBlock()).getName().endsWith(
-            oldBlock.getBlock().getGenerationStamp() + ".meta"));
+    assertEquals(utils.getStoredGenerationStamp(oldBlock.getBlock()),
+        oldBlock.getBlock().getGenerationStamp());
 
     // Validate the file
     FileStatus fileStatus = fs.getFileStatus(p);
@@ -812,18 +813,18 @@ public class TestFileTruncate {
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // Old replica is disregarded and replaced with the truncated one on dn0
-    assertEquals(cluster.getBlockFile(dn0, newBlock.getBlock()).length(), 
+    FsDatasetTestUtils utils = cluster.getFsDatasetTestUtils(dn0);
+    assertEquals(utils.getStoredDataLength(newBlock.getBlock()),
         newBlock.getBlockSize());
-    assertTrue(cluster.getBlockMetadataFile(dn0, 
-        newBlock.getBlock()).getName().endsWith(
-            newBlock.getBlock().getGenerationStamp() + ".meta"));
+    assertEquals(utils.getStoredGenerationStamp(newBlock.getBlock()),
+        newBlock.getBlock().getGenerationStamp());
 
     // Old replica is disregarded and replaced with the truncated one on dn1
-    assertEquals(cluster.getBlockFile(dn1, newBlock.getBlock()).length(), 
+    utils = cluster.getFsDatasetTestUtils(dn1);
+    assertEquals(utils.getStoredDataLength(newBlock.getBlock()),
         newBlock.getBlockSize());
-    assertTrue(cluster.getBlockMetadataFile(dn1, 
-        newBlock.getBlock()).getName().endsWith(
-            newBlock.getBlock().getGenerationStamp() + ".meta"));
+    assertEquals(utils.getStoredGenerationStamp(newBlock.getBlock()),
+        newBlock.getBlock().getGenerationStamp());
 
     // Validate the file
     FileStatus fileStatus = fs.getFileStatus(p);