HDFS-4268. Remove redundant enum NNHAStatusHeartbeat.State. Contributed by Konstantin Shvachko.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1417752 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
3337588975
commit
8bb0dc34e4
@ -402,6 +402,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
|
||||
|
@ -26,6 +26,7 @@
|
||||
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 static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
|
||||
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());
|
||||
}
|
||||
|
@ -26,6 +26,7 @@
|
||||
|
||||
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;
|
||||
@ -411,7 +412,7 @@ synchronized void updateActorStatesFromHeartbeat(
|
||||
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;
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
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.namenode.ha.HAState;
|
||||
import org.apache.hadoop.ipc.StandbyException;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class BackupState extends HAState {
|
||||
|
||||
public BackupState() {
|
||||
@ -26,7 +28,7 @@ public boolean shouldPopulateReplQueues() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Override // HAState
|
||||
public void enterState(HAContext context) throws ServiceFailedException {
|
||||
try {
|
||||
context.startActiveServices();
|
||||
@ -35,7 +37,7 @@ public void enterState(HAContext context) throws ServiceFailedException {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@Override // HAState
|
||||
public void exitState(HAContext context) throws ServiceFailedException {
|
||||
try {
|
||||
context.stopActiveServices();
|
||||
@ -44,7 +46,7 @@ public void exitState(HAContext context) throws ServiceFailedException {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@Override // HAState
|
||||
public void prepareToExitState(HAContext context) throws ServiceFailedException {
|
||||
context.prepareToStopStandbyServices();
|
||||
}
|
||||
|
@ -168,7 +168,6 @@
|
||||
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;
|
||||
@ -1004,7 +1003,7 @@ public boolean isInStandbyState() {
|
||||
return haEnabled;
|
||||
}
|
||||
|
||||
return haContext.getState() instanceof StandbyState;
|
||||
return HAServiceState.STANDBY == haContext.getState().getServiceState();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -3437,15 +3436,7 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
@ -3874,7 +3865,7 @@ private void enter() {
|
||||
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;
|
||||
@ -3917,7 +3908,8 @@ private synchronized void initializeReplQueues() {
|
||||
* initializing replication queues.
|
||||
*/
|
||||
private synchronized boolean canInitializeReplQueues() {
|
||||
return !isInStandbyState() && blockSafe >= blockReplQueueThreshold;
|
||||
return shouldPopulateReplQueues()
|
||||
&& blockSafe >= blockReplQueueThreshold;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -4257,7 +4249,7 @@ public boolean isInStartupSafeMode() {
|
||||
|
||||
@Override
|
||||
public boolean isPopulatingReplQueues() {
|
||||
if (isInStandbyState()) {
|
||||
if (!shouldPopulateReplQueues()) {
|
||||
return false;
|
||||
}
|
||||
// safeMode is volatile, and may be set to null at any time
|
||||
@ -4267,6 +4259,12 @@ public boolean isPopulatingReplQueues() {
|
||||
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
|
||||
|
@ -19,31 +19,26 @@
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
@ -29,6 +29,7 @@
|
||||
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.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 @@ private DatanodeProtocolClientSideTranslatorPB setupNNMock(int nnIdx)
|
||||
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 void testPickActiveNameNode() throws Exception {
|
||||
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 void testPickActiveNameNode() throws Exception {
|
||||
// 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());
|
||||
|
||||
|
@ -49,6 +49,7 @@
|
||||
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.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 DatanodeRegistration answer(InvocationOnMock invocation)
|
||||
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
|
||||
|
Loading…
Reference in New Issue
Block a user