|
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
|
|
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
|
|
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.junit.Assert;
|
|
|
import org.junit.BeforeClass;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
@@ -90,10 +91,10 @@ public class TestSCMSafeModeManager {
|
|
|
private void testSafeMode(int numContainers) throws Exception {
|
|
|
containers = new ArrayList<>();
|
|
|
containers.addAll(HddsTestUtils.getContainerInfo(numContainers));
|
|
|
- // Assign open state to containers to be included in the safe mode
|
|
|
- // container list
|
|
|
+
|
|
|
+ // Currently only considered containers which are not in open state.
|
|
|
for (ContainerInfo container : containers) {
|
|
|
- container.setState(HddsProtos.LifeCycleState.OPEN);
|
|
|
+ container.setState(HddsProtos.LifeCycleState.CLOSED);
|
|
|
}
|
|
|
scmSafeModeManager = new SCMSafeModeManager(
|
|
|
config, containers, null, queue);
|
|
@@ -101,15 +102,28 @@ public class TestSCMSafeModeManager {
|
|
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
|
|
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
|
HddsTestUtils.createNodeRegistrationContainerReport(containers));
|
|
|
+
|
|
|
+ long cutOff = (long) Math.ceil(numContainers * config.getDouble(
|
|
|
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT,
|
|
|
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT));
|
|
|
+
|
|
|
+ Assert.assertEquals(cutOff, scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getNumContainerWithOneReplicaReportedThreshold().value());
|
|
|
+
|
|
|
GenericTestUtils.waitFor(() -> {
|
|
|
return !scmSafeModeManager.getInSafeMode();
|
|
|
}, 100, 1000 * 5);
|
|
|
+
|
|
|
+ Assert.assertEquals(cutOff, scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentContainersWithOneReplicaReportedCount().value());
|
|
|
+
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testSafeModeExitRule() throws Exception {
|
|
|
containers = new ArrayList<>();
|
|
|
- containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
|
|
|
+ int numContainers = 100;
|
|
|
+ containers.addAll(HddsTestUtils.getContainerInfo(numContainers));
|
|
|
// Assign open state to containers to be included in the safe mode
|
|
|
// container list
|
|
|
for (ContainerInfo container : containers) {
|
|
@@ -118,15 +132,30 @@ public class TestSCMSafeModeManager {
|
|
|
scmSafeModeManager = new SCMSafeModeManager(
|
|
|
config, containers, null, queue);
|
|
|
|
|
|
+ long cutOff = (long) Math.ceil(numContainers * config.getDouble(
|
|
|
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT,
|
|
|
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT));
|
|
|
+
|
|
|
+ Assert.assertEquals(cutOff, scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getNumContainerWithOneReplicaReportedThreshold().value());
|
|
|
+
|
|
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
|
|
|
|
|
testContainerThreshold(containers.subList(0, 25), 0.25);
|
|
|
+ Assert.assertEquals(25, scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentContainersWithOneReplicaReportedCount().value());
|
|
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
|
|
testContainerThreshold(containers.subList(25, 50), 0.50);
|
|
|
+ Assert.assertEquals(50, scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentContainersWithOneReplicaReportedCount().value());
|
|
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
|
|
testContainerThreshold(containers.subList(50, 75), 0.75);
|
|
|
+ Assert.assertEquals(75, scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentContainersWithOneReplicaReportedCount().value());
|
|
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
|
|
testContainerThreshold(containers.subList(75, 100), 1.0);
|
|
|
+ Assert.assertEquals(100, scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentContainersWithOneReplicaReportedCount().value());
|
|
|
|
|
|
GenericTestUtils.waitFor(() -> {
|
|
|
return !scmSafeModeManager.getInSafeMode();
|
|
@@ -248,7 +277,6 @@ public class TestSCMSafeModeManager {
|
|
|
pipelineManager, queue);
|
|
|
|
|
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
|
|
-
|
|
|
testContainerThreshold(containers, 1.0);
|
|
|
|
|
|
List<Pipeline> pipelines = pipelineManager.getPipelines();
|
|
@@ -260,6 +288,14 @@ public class TestSCMSafeModeManager {
|
|
|
scmSafeModeManager.getOneReplicaPipelineSafeModeRule()
|
|
|
.getThresholdCount();
|
|
|
|
|
|
+ Assert.assertEquals(healthyPipelineThresholdCount,
|
|
|
+ scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getNumHealthyPipelinesThreshold().value());
|
|
|
+
|
|
|
+ Assert.assertEquals(oneReplicaThresholdCount,
|
|
|
+ scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getNumPipelinesWithAtleastOneReplicaReportedThreshold().value());
|
|
|
+
|
|
|
// Because even if no pipelines are there, and threshold we set to zero,
|
|
|
// we shall a get an event when datanode is registered. In that case,
|
|
|
// validate will return true, and add this to validatedRules.
|
|
@@ -273,13 +309,27 @@ public class TestSCMSafeModeManager {
|
|
|
|
|
|
if (i < healthyPipelineThresholdCount) {
|
|
|
checkHealthy(i + 1);
|
|
|
+ Assert.assertEquals(i + 1,
|
|
|
+ scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentHealthyPipelinesCount().value());
|
|
|
}
|
|
|
|
|
|
if (i < oneReplicaThresholdCount) {
|
|
|
checkOpen(i + 1);
|
|
|
+ Assert.assertEquals(i + 1,
|
|
|
+ scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentPipelinesWithAtleastOneReplicaCount().value());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ Assert.assertEquals(healthyPipelineThresholdCount,
|
|
|
+ scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentHealthyPipelinesCount().value());
|
|
|
+
|
|
|
+ Assert.assertEquals(oneReplicaThresholdCount,
|
|
|
+ scmSafeModeManager.getSafeModeMetrics()
|
|
|
+ .getCurrentPipelinesWithAtleastOneReplicaCount().value());
|
|
|
+
|
|
|
|
|
|
GenericTestUtils.waitFor(() -> {
|
|
|
return !scmSafeModeManager.getInSafeMode();
|