فهرست منبع

HDFS-5283. Under construction blocks only inside snapshots should not be counted in safemode threshhold. Contributed by Vinay

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1532857 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 سال پیش
والد
کامیت
ffe347151e

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

@@ -349,6 +349,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5352. Server#initLog() doesn't close InputStream in httpfs. (Ted Yu via
     jing9)
 
+    HDFS-5283. Under construction blocks only inside snapshots should not be
+    counted in safemode threshhold.  (Vinay via szetszwo)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1780,6 +1780,14 @@ public class BlockManager {
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
         ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
             node, iblk, reportedState);
+        // OpenFileBlocks only inside snapshots also will be added to safemode
+        // threshold. So we need to update such blocks to safemode
+        // refer HDFS-5283
+        BlockInfoUnderConstruction blockUC = (BlockInfoUnderConstruction) storedBlock;
+        if (namesystem.isInSnapshot(blockUC)) {
+          int numOfReplicas = blockUC.getNumExpectedLocations();
+          namesystem.incrementSafeBlockCount(numOfReplicas);
+        }
         //and fall through to next clause
       }      
       //add replica if appropriate

+ 34 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -165,6 +165,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -3717,6 +3718,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   BlockInfo getStoredBlock(Block block) {
     return blockManager.getStoredBlock(block);
   }
+  
+  @Override
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
+    assert hasReadOrWriteLock();
+    final BlockCollection bc = blockUC.getBlockCollection();
+    if (bc == null || !(bc instanceof INodeFileUnderConstruction)) {
+      return false;
+    }
+
+    INodeFileUnderConstruction inodeUC = (INodeFileUnderConstruction) blockUC
+        .getBlockCollection();
+    String fullName = inodeUC.getName();
+    try {
+      if (fullName != null && fullName.startsWith(Path.SEPARATOR)
+          && dir.getINode(fullName) == inodeUC) {
+        // If file exists in normal path then no need to look in snapshot
+        return false;
+      }
+    } catch (UnresolvedLinkException e) {
+      LOG.error("Error while resolving the link : " + fullName, e);
+      return false;
+    }
+    /*
+     * 1. if bc is an instance of INodeFileUnderConstructionWithSnapshot, and
+     * bc is not in the current fsdirectory tree, bc must represent a snapshot
+     * file. 
+     * 2. if fullName is not an absolute path, bc cannot be existent in the 
+     * current fsdirectory tree. 
+     * 3. if bc is not the current node associated with fullName, bc must be a
+     * snapshot inode.
+     */
+    return true;
+  }
 
   void commitBlockSynchronization(ExtendedBlock lastblock,
       long newgenerationstamp, long newlength,

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ipc.StandbyException;
@@ -43,4 +44,6 @@ public interface Namesystem extends RwLock, SafeMode {
   public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
 
   public void checkOperation(OperationCategory read) throws StandbyException;
+
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
 }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -994,4 +994,8 @@ public class DFSTestUtil {
           cluster.getNameNodeRpc(nnIndex), filePath, 0L, bytes.length);
     } while (locatedBlocks.isUnderConstruction());
   }
+
+  public static void abortStream(DFSOutputStream out) throws IOException {
+    out.abort();
+  }
 }