Browse Source

HDFS-1522. Combine two BLOCK_FILE_PREFIX constants into one. Contributed by Dongming Liang.

Dongming Liang 10 years ago
parent
commit
ada545d3b7

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

@@ -401,6 +401,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7535. Utilize Snapshot diff report for distcp. (jing9)
 
+    HDFS-1522. Combine two BLOCK_FILE_PREFIX constants into one.
+    (Dongming Liang via shv)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
@@ -351,7 +352,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       DatanodeID datanodeID, PeerCache peerCache) {
     // Path is used only for printing block and file information in debug
-    super(new Path("/blk_" + blockId + ":" + bpid + ":of:"+ file)/*too non path-like?*/,
+    super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
+                    ":" + bpid + ":of:"+ file)/*too non path-like?*/,
           1, verifyChecksum,
           checksum.getChecksumSize() > 0? checksum : null, 
           checksum.getBytesPerChecksum(),

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -83,7 +83,6 @@ import java.util.concurrent.Future;
 public class DataStorage extends Storage {
 
   public final static String BLOCK_SUBDIR_PREFIX = "subdir";
-  final static String BLOCK_FILE_PREFIX = "blk_";
   final static String COPY_FILE_PREFIX = "dncp_";
   final static String STORAGE_DIR_DETACHED = "detach";
   public final static String STORAGE_DIR_RBW = "rbw";
@@ -1250,7 +1249,7 @@ public class DataStorage extends Storage {
     String[] blockNames = from.list(new java.io.FilenameFilter() {
       @Override
       public boolean accept(File dir, String name) {
-        return name.startsWith(BLOCK_FILE_PREFIX);
+        return name.startsWith(Block.BLOCK_FILE_PREFIX);
       }
     });
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -633,7 +633,7 @@ public class DirectoryScanner implements Runnable {
           continue;
         }
         if (!Block.isBlockFilename(files[i])) {
-          if (isBlockMetaFile("blk_", files[i].getName())) {
+          if (isBlockMetaFile(Block.BLOCK_FILE_PREFIX, files[i].getName())) {
             long blockId = Block.getBlockId(files[i].getName());
             verifyFileLocation(files[i].getParentFile(), bpFinalizedDir,
                 blockId);

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

@@ -430,7 +430,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
 
     @Override
     public boolean accept(File dir, String name) {
-      return !name.endsWith(".meta") && name.startsWith("blk_");
+      return !name.endsWith(".meta") &&
+              name.startsWith(Block.BLOCK_FILE_PREFIX);
     }
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -299,7 +299,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         out.println(sb.toString());
         sb.append(" for blockIds: \n");
         for (String blk: blocks) {
-          if(blk == null || !blk.contains("blk_")) {
+          if(blk == null || !blk.contains(Block.BLOCK_FILE_PREFIX)) {
             out.println("Incorrect blockId format: " + blk);
             continue;
           }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -2567,8 +2567,8 @@ public class MiniDFSCluster {
       return null;
     }
     for (File f : files) {
-      if (f.getName().startsWith("blk_") && f.getName().endsWith(
-          Block.METADATA_EXTENSION)) {
+      if (f.getName().startsWith(Block.BLOCK_FILE_PREFIX) &&
+              f.getName().endsWith(Block.METADATA_EXTENSION)) {
         results.add(f);
       } else if (f.isDirectory()) {
         List<File> subdirResults = getAllBlockMetadataFiles(f);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClientFaultInjector;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Before;
@@ -176,7 +177,7 @@ public class TestCrcCorruption {
       assertTrue("Blocks do not exist in data-dir", (blocks != null) && (blocks.length > 0));
       int num = 0;
       for (int idx = 0; idx < blocks.length; idx++) {
-        if (blocks[idx].getName().startsWith("blk_") &&
+        if (blocks[idx].getName().startsWith(Block.BLOCK_FILE_PREFIX) &&
             blocks[idx].getName().endsWith(".meta")) {
           num++;
           if (num % 3 == 0) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java

@@ -77,7 +77,7 @@ public class TestFileCorruption {
       File[] blocks = data_dir.listFiles();
       assertTrue("Blocks do not exist in data-dir", (blocks != null) && (blocks.length > 0));
       for (int idx = 0; idx < blocks.length; idx++) {
-        if (!blocks[idx].getName().startsWith("blk_")) {
+        if (!blocks[idx].getName().startsWith(Block.BLOCK_FILE_PREFIX)) {
           continue;
         }
         System.out.println("Deliberately removing file "+blocks[idx].getName());

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.util.Time;
 import org.junit.Test;
@@ -520,7 +521,7 @@ public class TestReplication {
       String blockFile = null;
       File[] listFiles = participatedNodeDirs.listFiles();
       for (File file : listFiles) {
-        if (file.getName().startsWith("blk_")
+        if (file.getName().startsWith(Block.BLOCK_FILE_PREFIX)
             && !file.getName().endsWith("meta")) {
           blockFile = file.getName();
           for (File file1 : nonParticipatedNodeDirs) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -319,7 +319,7 @@ public class TestDataNodeVolumeFailure {
   private boolean deteteBlocks(File dir) {
     File [] fileList = dir.listFiles();
     for(File f : fileList) {
-      if(f.getName().startsWith("blk_")) {
+      if(f.getName().startsWith(Block.BLOCK_FILE_PREFIX)) {
         if(!f.delete())
           return false;