HDDS-797. If DN is started before SCM, it does not register. Contributed by Hanisha Koneru.
This commit is contained in:
parent
15df2e7a75
commit
c8ca1747c0
|
@ -64,6 +64,8 @@ public class VersionEndpointTask implements
|
|||
public EndpointStateMachine.EndPointStates call() throws Exception {
|
||||
rpcEndPoint.lock();
|
||||
try{
|
||||
if (rpcEndPoint.getState().equals(
|
||||
EndpointStateMachine.EndPointStates.GETVERSION)) {
|
||||
SCMVersionResponseProto versionResponse =
|
||||
rpcEndPoint.getEndPoint().getVersion(null);
|
||||
VersionResponse response = VersionResponse.getFromProtobuf(
|
||||
|
@ -85,6 +87,7 @@ public class VersionEndpointTask implements
|
|||
"cannot be null");
|
||||
|
||||
// If version file does not exist create version file and also set scmId
|
||||
|
||||
for (Map.Entry<String, HddsVolume> entry : volumeMap.entrySet()) {
|
||||
HddsVolume hddsVolume = entry.getValue();
|
||||
boolean result = HddsVolumeUtil.checkVolume(hddsVolume, scmId,
|
||||
|
@ -108,6 +111,10 @@ public class VersionEndpointTask implements
|
|||
rpcEndPoint.getState().getNextState();
|
||||
rpcEndPoint.setState(nextState);
|
||||
rpcEndPoint.zeroMissedCount();
|
||||
} else {
|
||||
LOG.debug("Cannot execute GetVersion task as endpoint state machine " +
|
||||
"is in {} state", rpcEndPoint.getState());
|
||||
}
|
||||
} catch (DiskOutOfSpaceException ex) {
|
||||
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
|
||||
} catch(IOException ex) {
|
||||
|
|
|
@ -23,9 +23,13 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
|
|||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
|
||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
|
||||
import org.apache.hadoop.ozone.container.common.statemachine
|
||||
.DatanodeStateMachine;
|
||||
import org.apache.hadoop.ozone.container.common.statemachine
|
||||
.EndpointStateMachine;
|
||||
import org.apache.hadoop.ozone.container.ozoneimpl.TestOzoneContainer;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||
|
@ -214,4 +218,50 @@ public class TestMiniOzoneCluster {
|
|||
out.write("malformed".getBytes());
|
||||
out.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that a DN can register with SCM even if it was started before the SCM.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout = 300_000)
|
||||
public void testDNstartAfterSCM() throws Exception {
|
||||
// Start a cluster with 1 DN
|
||||
cluster = MiniOzoneCluster.newBuilder(conf)
|
||||
.setNumDatanodes(1)
|
||||
.build();
|
||||
cluster.waitForClusterToBeReady();
|
||||
|
||||
// Stop the SCM
|
||||
StorageContainerManager scm = cluster.getStorageContainerManager();
|
||||
scm.stop();
|
||||
|
||||
// Restart DN
|
||||
cluster.restartHddsDatanode(0, false);
|
||||
|
||||
// DN should be in GETVERSION state till the SCM is restarted.
|
||||
// Check DN endpoint state for 20 seconds
|
||||
DatanodeStateMachine dnStateMachine = cluster.getHddsDatanodes().get(0)
|
||||
.getDatanodeStateMachine();
|
||||
for (int i = 0; i < 20; i++) {
|
||||
for (EndpointStateMachine endpoint :
|
||||
dnStateMachine.getConnectionManager().getValues()) {
|
||||
Assert.assertEquals(
|
||||
EndpointStateMachine.EndPointStates.GETVERSION,
|
||||
endpoint.getState());
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
// DN should successfully register with the SCM after SCM is restarted.
|
||||
// Restart the SCM
|
||||
cluster.restartStorageContainerManager();
|
||||
// Wait for DN to register
|
||||
cluster.waitForClusterToBeReady();
|
||||
// DN should be in HEARTBEAT state after registering with the SCM
|
||||
for (EndpointStateMachine endpoint :
|
||||
dnStateMachine.getConnectionManager().getValues()) {
|
||||
Assert.assertEquals(EndpointStateMachine.EndPointStates.HEARTBEAT,
|
||||
endpoint.getState());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue