You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sh...@apache.org on 2012/12/06 08:23:42 UTC

svn commit: r1417753 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/protocolPB/ src/main/java/org/apache/hadoop/hdfs/server/datanode/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ ...

Author: shv
Date: Thu Dec  6 07:23:41 2012
New Revision: 1417753

URL: http://svn.apache.org/viewvc?rev=1417753&view=rev
Log:
HDFS-4268. Remove redundant enum NNHAStatusHeartbeat.State. Contributed by Konstantin Shvachko.

Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Thu Dec  6 07:23:41 2012
@@ -143,6 +143,8 @@ Release 2.0.3-alpha - Unreleased
 
     HDFS-3935. Add JournalNode to the start/stop scripts (Andy Isaacson via todd)
 
+    HDFS-4268. Remove redundant enum NNHAStatusHeartbeat.State. (shv)
+
   OPTIMIZATIONS
 
   BUG FIXES 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java Thu Dec  6 07:23:41 2012
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.ContentSumma
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -1232,9 +1233,9 @@ public class PBHelper {
     if (s == null) return null;
     switch (s.getState()) {
     case ACTIVE:
-      return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.ACTIVE, s.getTxid());
+      return new NNHAStatusHeartbeat(HAServiceState.ACTIVE, s.getTxid());
     case STANDBY:
-      return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.STANDBY, s.getTxid());
+      return new NNHAStatusHeartbeat(HAServiceState.STANDBY, s.getTxid());
     default:
       throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" + s.getState());
     }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java Thu Dec  6 07:23:41 2012
@@ -26,6 +26,7 @@ import java.util.concurrent.CopyOnWriteA
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -412,7 +413,7 @@ class BPOfferService {
     final long txid = nnHaState.getTxId();
     
     final boolean nnClaimsActive =
-      nnHaState.getState() == NNHAStatusHeartbeat.State.ACTIVE;
+      nnHaState.getState() == HAServiceState.ACTIVE;
     final boolean bposThinksActive = bpServiceToActive == actor;
     final boolean isMoreRecentClaim = txid > lastActiveClaimTxId; 
     

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java Thu Dec  6 07:23:41 2012
@@ -2,6 +2,7 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.IOException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
@@ -9,6 +10,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.ipc.StandbyException;
 
+@InterfaceAudience.Private
 public class BackupState extends HAState {
 
   public BackupState() {
@@ -26,7 +28,7 @@ public class BackupState extends HAState
     return false;
   }
 
-  @Override
+  @Override // HAState
   public void enterState(HAContext context) throws ServiceFailedException {
     try {
       context.startActiveServices();
@@ -35,7 +37,7 @@ public class BackupState extends HAState
     }
   }
 
-  @Override
+  @Override // HAState
   public void exitState(HAContext context) throws ServiceFailedException {
     try {
       context.stopActiveServices();
@@ -44,7 +46,7 @@ public class BackupState extends HAState
     }
   }
 
-  @Override
+  @Override // HAState
   public void prepareToExitState(HAContext context) throws ServiceFailedException {
     context.prepareToStopStandbyServices();
   }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Thu Dec  6 07:23:41 2012
@@ -166,7 +166,6 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
-import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -986,8 +985,8 @@ public class FSNamesystem implements Nam
       // start in active.
       return haEnabled;
     }
-  
-    return haContext.getState() instanceof StandbyState;
+
+    return HAServiceState.STANDBY == haContext.getState().getServiceState();
   }
 
   /**
@@ -3412,15 +3411,7 @@ public class FSNamesystem implements Nam
 
   private NNHAStatusHeartbeat createHaStatusHeartbeat() {
     HAState state = haContext.getState();
-    NNHAStatusHeartbeat.State hbState;
-    if (state.getServiceState() == HAServiceState.ACTIVE) {
-      hbState = NNHAStatusHeartbeat.State.ACTIVE;
-    } else if (state.getServiceState() == HAServiceState.STANDBY) {
-      hbState = NNHAStatusHeartbeat.State.STANDBY;      
-    } else {
-      throw new AssertionError("Invalid state: " + state.getClass());
-    }
-    return new NNHAStatusHeartbeat(hbState,
+    return new NNHAStatusHeartbeat(state.getServiceState(),
         getFSImage().getLastAppliedOrWrittenTxId());
   }
 
@@ -3849,7 +3840,7 @@ public class FSNamesystem implements Nam
     private synchronized void leave() {
       // if not done yet, initialize replication queues.
       // In the standby, do not populate repl queues
-      if (!isPopulatingReplQueues() && !isInStandbyState()) {
+      if (!isPopulatingReplQueues() && shouldPopulateReplQueues()) {
         initializeReplQueues();
       }
       long timeInSafemode = now() - startTime;
@@ -3892,7 +3883,8 @@ public class FSNamesystem implements Nam
      * initializing replication queues.
      */
     private synchronized boolean canInitializeReplQueues() {
-      return !isInStandbyState() && blockSafe >= blockReplQueueThreshold;
+      return shouldPopulateReplQueues()
+          && blockSafe >= blockReplQueueThreshold;
     }
       
     /** 
@@ -4232,7 +4224,7 @@ public class FSNamesystem implements Nam
 
   @Override
   public boolean isPopulatingReplQueues() {
-    if (isInStandbyState()) {
+    if (!shouldPopulateReplQueues()) {
       return false;
     }
     // safeMode is volatile, and may be set to null at any time
@@ -4241,7 +4233,13 @@ public class FSNamesystem implements Nam
       return true;
     return safeMode.isPopulatingReplQueues();
   }
-    
+
+  private boolean shouldPopulateReplQueues() {
+    if(haContext == null || haContext.getState() == null)
+      return false;
+    return haContext.getState().shouldPopulateReplQueues();
+  }
+
   @Override
   public void incrementSafeBlockCount(int replication) {
     // safeMode is volatile, and may be set to null at any time

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java Thu Dec  6 07:23:41 2012
@@ -19,31 +19,26 @@ package org.apache.hadoop.hdfs.server.pr
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class NNHAStatusHeartbeat {
 
-  private State state;
+  private HAServiceState state;
   private long txid = HdfsConstants.INVALID_TXID;
   
-  public NNHAStatusHeartbeat(State state, long txid) {
+  public NNHAStatusHeartbeat(HAServiceState state, long txid) {
     this.state = state;
     this.txid = txid;
   }
 
-  public State getState() {
+  public HAServiceState getState() {
     return state;
   }
   
   public long getTxId() {
     return txid;
   }
-  
-  @InterfaceAudience.Private
-  public enum State {
-    ACTIVE,
-    STANDBY;
-  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java Thu Dec  6 07:23:41 2012
@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -41,7 +42,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
-import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat.State;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@@ -123,7 +123,7 @@ public class TestBPOfferService {
           Mockito.anyInt(),
           Mockito.anyInt(),
           Mockito.anyInt());
-    mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(State.STANDBY, 0);
+    mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     return mock;
   }
   
@@ -255,12 +255,12 @@ public class TestBPOfferService {
       assertNull(bpos.getActiveNN());
 
       // Have NN1 claim active at txid 1
-      mockHaStatuses[0] = new NNHAStatusHeartbeat(State.ACTIVE, 1);
+      mockHaStatuses[0] = new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1);
       bpos.triggerHeartbeatForTests();
       assertSame(mockNN1, bpos.getActiveNN());
 
       // NN2 claims active at a higher txid
-      mockHaStatuses[1] = new NNHAStatusHeartbeat(State.ACTIVE, 2);
+      mockHaStatuses[1] = new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 2);
       bpos.triggerHeartbeatForTests();
       assertSame(mockNN2, bpos.getActiveNN());
       
@@ -272,12 +272,12 @@ public class TestBPOfferService {
       // Even if NN2 goes to standby, DN shouldn't reset to talking to NN1,
       // because NN1's txid is lower than the last active txid. Instead,
       // it should consider neither active.
-      mockHaStatuses[1] = new NNHAStatusHeartbeat(State.STANDBY, 2);
+      mockHaStatuses[1] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 2);
       bpos.triggerHeartbeatForTests();
       assertNull(bpos.getActiveNN());
       
       // Now if NN1 goes back to a higher txid, it should be considered active
-      mockHaStatuses[0] = new NNHAStatusHeartbeat(State.ACTIVE, 3);
+      mockHaStatuses[0] = new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 3);
       bpos.triggerHeartbeatForTests();
       assertSame(mockNN1, bpos.getActiveNN());
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java?rev=1417753&r1=1417752&r2=1417753&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java Thu Dec  6 07:23:41 2012
@@ -49,6 +49,7 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -72,7 +73,6 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
-import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat.State;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -157,7 +157,7 @@ public class TestBlockRecovery {
             Mockito.anyInt()))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
-            new NNHAStatusHeartbeat(State.ACTIVE, 1)));
+            new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1)));
 
     dn = new DataNode(conf, dirs, null) {
       @Override