瀏覽代碼

HDFS-5014. Process register commands with out holding BPOfferService lock. Contributed by Vinay.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1543861 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G 11 年之前
父節點
當前提交
04cf2a768c

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

@@ -692,6 +692,9 @@ Release 2.2.1 - UNRELEASED
     HDFS-4516. Client crash after block allocation and NN switch before lease recovery for 
     the same file can cause readers to fail forever (VinaayKumar B via umamahesh)
 
+    HDFS-5014. Process register commands with out holding BPOfferService lock. 
+    (Vinaykumar B via umamahesh)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 31 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -73,7 +73,7 @@ class BPOfferService {
    * This is assigned after the second phase of the
    * handshake.
    */
-  DatanodeRegistration bpRegistration;
+  volatile DatanodeRegistration bpRegistration;
   
   private final DataNode dn;
 
@@ -295,7 +295,7 @@ class BPOfferService {
    * NN, it calls this function to verify that the NN it connected to
    * is consistent with other NNs serving the block-pool.
    */
-  void registrationSucceeded(BPServiceActor bpServiceActor,
+  synchronized void registrationSucceeded(BPServiceActor bpServiceActor,
       DatanodeRegistration reg) throws IOException {
     if (bpRegistration != null) {
       checkNSEquality(bpRegistration.getStorageInfo().getNamespaceID(),
@@ -497,17 +497,37 @@ class BPOfferService {
     }
   }
 
-  synchronized boolean processCommandFromActor(DatanodeCommand cmd,
+  boolean processCommandFromActor(DatanodeCommand cmd,
       BPServiceActor actor) throws IOException {
     assert bpServices.contains(actor);
-    if (actor == bpServiceToActive) {
-      return processCommandFromActive(cmd, actor);
-    } else {
-      return processCommandFromStandby(cmd, actor);
+    if (cmd == null) {
+      return true;
+    }
+    /*
+     * Datanode Registration can be done asynchronously here. No need to hold
+     * the lock. for more info refer HDFS-5014
+     */
+    if (DatanodeProtocol.DNA_REGISTER == cmd.getAction()) {
+      // namenode requested a registration - at start or if NN lost contact
+      // Just logging the claiming state is OK here instead of checking the
+      // actor state by obtaining the lock
+      LOG.info("DatanodeCommand action : DNA_REGISTER from " + actor.nnAddr
+          + " with " + actor.state + " state");
+      actor.reRegister();
+      return true;
+    }
+    synchronized (this) {
+      if (actor == bpServiceToActive) {
+        return processCommandFromActive(cmd, actor);
+      } else {
+        return processCommandFromStandby(cmd, actor);
+      }
     }
   }
 
   /**
+   * This method should handle all commands from Active namenode except
+   * DNA_REGISTER which should be handled earlier itself.
    * 
    * @param cmd
    * @return true if further processing may be required or false otherwise. 
@@ -515,8 +535,6 @@ class BPOfferService {
    */
   private boolean processCommandFromActive(DatanodeCommand cmd,
       BPServiceActor actor) throws IOException {
-    if (cmd == null)
-      return true;
     final BlockCommand bcmd = 
       cmd instanceof BlockCommand? (BlockCommand)cmd: null;
     final BlockIdCommand blockIdCmd = 
@@ -560,11 +578,6 @@ class BPOfferService {
       // TODO: DNA_SHUTDOWN appears to be unused - the NN never sends this command
       // See HDFS-2987.
       throw new UnsupportedOperationException("Received unimplemented DNA_SHUTDOWN");
-    case DatanodeProtocol.DNA_REGISTER:
-      // namenode requested a registration - at start or if NN lost contact
-      LOG.info("DatanodeCommand action: DNA_REGISTER");
-      actor.reRegister();
-      break;
     case DatanodeProtocol.DNA_FINALIZE:
       String bp = ((FinalizeCommand) cmd).getBlockPoolId(); 
       assert getBlockPoolId().equals(bp) :
@@ -604,16 +617,13 @@ class BPOfferService {
     return true;
   }
  
+  /**
+   * This method should handle commands from Standby namenode except
+   * DNA_REGISTER which should be handled earlier itself.
+   */
   private boolean processCommandFromStandby(DatanodeCommand cmd,
       BPServiceActor actor) throws IOException {
-    if (cmd == null)
-      return true;
     switch(cmd.getAction()) {
-    case DatanodeProtocol.DNA_REGISTER:
-      // namenode requested a registration - at start or if NN lost contact
-      LOG.info("DatanodeCommand action from standby: DNA_REGISTER");
-      actor.reRegister();
-      break;
     case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
       LOG.info("DatanodeCommand action from standby: DNA_ACCESSKEYUPDATE");
       if (dn.isBlockTokenEnabled) {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -29,6 +29,7 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -73,6 +74,7 @@ class BPServiceActor implements Runnable {
   
   static final Log LOG = DataNode.LOG;
   final InetSocketAddress nnAddr;
+  HAServiceState state;
 
   BPOfferService bpos;
   
@@ -569,6 +571,7 @@ class BPServiceActor implements Runnable {
             // that we should actually process.
             bpos.updateActorStatesFromHeartbeat(
                 this, resp.getNameNodeHaState());
+            state = resp.getNameNodeHaState().getState();
 
             long startProcessCommands = now();
             if (!processCommand(resp.getCommands()))