Browse Source

HADOOP-2148. Eliminate redundant data-node blockMap lookups. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@639044 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 17 years ago
parent
commit
dcc44b145f

+ 3 - 1
CHANGES.txt

@@ -126,10 +126,12 @@ Trunk (unreleased changes)
     HADOOP-2423.  Code optimization in FSNamesystem.mkdirs.
     (Tsz Wo (Nicholas), SZE via dhruba)
 
-		HADOOP-2606. ReplicationMonitor selects data-nodes to replicate directly
+    HADOOP-2606. ReplicationMonitor selects data-nodes to replicate directly
 		from needed replication blocks instead of looking up for the blocks for 
 		each live data-node. (shv)
 
+    HADOOP-2148. Eliminate redundant data-node blockMap lookups. (shv)
+
   BUG FIXES
 
     HADOOP-2195. '-mkdir' behaviour is now closer to Linux shell in case of

+ 0 - 13
src/java/org/apache/hadoop/dfs/BlockCrcUpgrade.java

@@ -441,11 +441,6 @@ class BlockCrcUpgradeUtils {
     Block block = blockInfo.block;
     FSDataset data = (FSDataset) blockInfo.dataNode.data;
     File blockFile = data.getBlockFile( block );
-    if ( blockFile == null || !blockFile.exists() ) {
-      throw new IOException("Block file "  + 
-                            ((blockFile != null) ? blockFile.getAbsolutePath()
-                              : "NULL") + " does not exist.");
-    }
     
     byte[] blockBuf = new byte[bytesPerChecksum];
     FileInputStream in = null;
@@ -785,9 +780,6 @@ class BlockCrcUpgradeUtils {
     Block block = blockInfo.block;
     FSDataset data = (FSDataset) blockInfo.dataNode.data;
     File blockFile = data.getBlockFile( block );
-    if (blockFile == null || !blockFile.exists()) {
-      throw new IOException("Could not local file for block");
-    }
     long blockLen = blockFile.length();
     if ( blockLen != blockFile.length()) {
       LOG.warn("Mismatch in length for block: local file size is " +
@@ -1465,11 +1457,6 @@ class BlockCrcUpgradeObjectDatanode extends UpgradeObjectDatanode {
         errors++;
         continue;
       }
-      if (!blockFile.exists()) {
-        LOG.error("could not find block file " + blockFile);
-        errors++;
-        continue;
-      }
       File metaFile = FSDataset.getMetaFile( blockFile );
       if ( metaFile.exists() ) {
         blocksPreviouslyUpgraded++;

+ 23 - 8
src/java/org/apache/hadoop/dfs/FSDataset.java

@@ -527,15 +527,16 @@ class FSDataset implements FSConstants, FSDatasetInterface {
   
   protected File getMetaFile(Block b) throws IOException {
     File blockFile = getBlockFile( b );
-    return new File( blockFile.getAbsolutePath() + METADATA_EXTENSION ); 
+    return getMetaFile(blockFile); 
   }
+
   public boolean metaFileExists(Block b) throws IOException {
     return getMetaFile(b).exists();
   }
   
   public long getMetaDataLength(Block b) throws IOException {
     File checksumFile = getMetaFile( b );
-  return checksumFile.length();
+    return checksumFile.length();
   }
 
   public MetaDataInputStream getMetaDataInputStream(Block b)
@@ -566,7 +567,7 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     volumeMap = new HashMap<Block, DatanodeBlockInfo>();
     volumes.getVolumeMap(volumeMap);
     blockWriteTimeout = Math.max(
-         conf.getInt("dfs.datanode.block.write.timeout.sec", 3600), 1) * 1000;
+         conf.getInt("dfs.datanode.block.write.timeout.sec", 3600), 1) * 1000L;
     registerMBean(storage.getStorageID());
   }
 
@@ -595,10 +596,10 @@ class FSDataset implements FSConstants, FSDatasetInterface {
    * Find the block's on-disk length
    */
   public long getLength(Block b) throws IOException {
-    if (!isValidBlock(b)) {
+    File f = validateBlockFile(b);
+    if(f == null) {
       throw new IOException("Block " + b + " is not valid.");
     }
-    File f = getFile(b);
     return f.length();
   }
 
@@ -606,10 +607,11 @@ class FSDataset implements FSConstants, FSDatasetInterface {
    * Get File name for a given block.
    */
   protected synchronized File getBlockFile(Block b) throws IOException {
-    if (!isValidBlock(b)) {
+    File f = validateBlockFile(b);
+    if(f == null) {
       throw new IOException("Block " + b + " is not valid.");
     }
-    return getFile(b);
+    return f;
   }
   
   public synchronized InputStream getBlockInputStream(Block b) throws IOException {
@@ -838,9 +840,18 @@ class FSDataset implements FSConstants, FSDatasetInterface {
    * Check whether the given block is a valid one.
    */
   public boolean isValidBlock(Block b) {
+    return validateBlockFile(b) != null;
+  }
+
+  /**
+   * Find the file corresponding to the block and return it if it exists.
+   */
+  File validateBlockFile(Block b) {
     //Should we check for metadata file too?
     File f = getFile(b);
-    return (f!= null && f.exists());
+    if(f != null && f.exists())
+      return f;
+    return null;
   }
 
   /**
@@ -973,6 +984,10 @@ class FSDataset implements FSConstants, FSDatasetInterface {
     return toString();
   }
   
+  /**
+   *  A duplicate of {@link #getLength()}
+   */
+  @Deprecated
   public long getBlockSize(Block b) {
     return getFile(b).length();
   }