|
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.chillmode;
|
|
|
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.util.ArrayList;
|
|
@@ -37,12 +38,17 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.*;
|
|
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
|
|
|
+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.BeforeClass;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.rules.TemporaryFolder;
|
|
|
import org.junit.rules.Timeout;
|
|
|
import org.mockito.Mockito;
|
|
|
|
|
@@ -56,7 +62,10 @@ public class TestSCMChillModeManager {
|
|
|
private List<ContainerInfo> containers;
|
|
|
|
|
|
@Rule
|
|
|
- public Timeout timeout = new Timeout(1000 * 35);
|
|
|
+ public Timeout timeout = new Timeout(1000 * 300);
|
|
|
+
|
|
|
+ @Rule
|
|
|
+ public final TemporaryFolder tempDir = new TemporaryFolder();
|
|
|
|
|
|
@BeforeClass
|
|
|
public static void setUp() {
|
|
@@ -124,6 +133,185 @@ public class TestSCMChillModeManager {
|
|
|
}, 100, 1000 * 5);
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ private OzoneConfiguration createConf(double healthyPercent,
|
|
|
+ double oneReplicaPercent) throws Exception {
|
|
|
+ OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
+ conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
|
|
|
+ tempDir.newFolder().toString());
|
|
|
+ conf.setBoolean(
|
|
|
+ HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
|
|
|
+ true);
|
|
|
+ conf.setDouble(HddsConfigKeys.
|
|
|
+ HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT, healthyPercent);
|
|
|
+ conf.setDouble(HddsConfigKeys.
|
|
|
+ HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT, oneReplicaPercent);
|
|
|
+
|
|
|
+ return conf;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testChillModeExitRuleWithPipelineAvailabilityCheck()
|
|
|
+ throws Exception{
|
|
|
+ testChillModeExitRuleWithPipelineAvailabilityCheck(100, 30, 8, 0.90, 1);
|
|
|
+ testChillModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0.10, 0.9);
|
|
|
+ testChillModeExitRuleWithPipelineAvailabilityCheck(100, 30, 8, 0, 0.9);
|
|
|
+ testChillModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0, 0);
|
|
|
+ testChillModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0, 0.5);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testFailWithIncorrectValueForHealthyPipelinePercent()
|
|
|
+ throws Exception {
|
|
|
+ try {
|
|
|
+ OzoneConfiguration conf = createConf(100,
|
|
|
+ 0.9);
|
|
|
+ MockNodeManager mockNodeManager = new MockNodeManager(true, 10);
|
|
|
+ PipelineManager pipelineManager = new SCMPipelineManager(conf,
|
|
|
+ mockNodeManager, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ conf, containers, pipelineManager, queue);
|
|
|
+ fail("testFailWithIncorrectValueForHealthyPipelinePercent");
|
|
|
+ } catch (IllegalArgumentException ex) {
|
|
|
+ GenericTestUtils.assertExceptionContains("value should be >= 0.0 and <=" +
|
|
|
+ " 1.0", ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testFailWithIncorrectValueForOneReplicaPipelinePercent()
|
|
|
+ throws Exception {
|
|
|
+ try {
|
|
|
+ OzoneConfiguration conf = createConf(0.9,
|
|
|
+ 200);
|
|
|
+ MockNodeManager mockNodeManager = new MockNodeManager(true, 10);
|
|
|
+ PipelineManager pipelineManager = new SCMPipelineManager(conf,
|
|
|
+ mockNodeManager, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ conf, containers, pipelineManager, queue);
|
|
|
+ fail("testFailWithIncorrectValueForOneReplicaPipelinePercent");
|
|
|
+ } catch (IllegalArgumentException ex) {
|
|
|
+ GenericTestUtils.assertExceptionContains("value should be >= 0.0 and <=" +
|
|
|
+ " 1.0", ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testFailWithIncorrectValueForChillModePercent() throws Exception {
|
|
|
+ try {
|
|
|
+ OzoneConfiguration conf = createConf(0.9, 0.1);
|
|
|
+ conf.setDouble(HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT, -1.0);
|
|
|
+ MockNodeManager mockNodeManager = new MockNodeManager(true, 10);
|
|
|
+ PipelineManager pipelineManager = new SCMPipelineManager(conf,
|
|
|
+ mockNodeManager, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ conf, containers, pipelineManager, queue);
|
|
|
+ fail("testFailWithIncorrectValueForChillModePercent");
|
|
|
+ } catch (IllegalArgumentException ex) {
|
|
|
+ GenericTestUtils.assertExceptionContains("value should be >= 0.0 and <=" +
|
|
|
+ " 1.0", ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ public void testChillModeExitRuleWithPipelineAvailabilityCheck(
|
|
|
+ int containerCount, int nodeCount, int pipelineCount,
|
|
|
+ double healthyPipelinePercent, double oneReplicaPercent)
|
|
|
+ throws Exception {
|
|
|
+
|
|
|
+ OzoneConfiguration conf = createConf(healthyPipelinePercent,
|
|
|
+ oneReplicaPercent);
|
|
|
+
|
|
|
+ containers = new ArrayList<>();
|
|
|
+ containers.addAll(HddsTestUtils.getContainerInfo(containerCount));
|
|
|
+
|
|
|
+ MockNodeManager mockNodeManager = new MockNodeManager(true, nodeCount);
|
|
|
+ SCMPipelineManager pipelineManager = new SCMPipelineManager(conf,
|
|
|
+ mockNodeManager, queue);
|
|
|
+ PipelineProvider mockRatisProvider =
|
|
|
+ new MockRatisPipelineProvider(mockNodeManager,
|
|
|
+ pipelineManager.getStateManager(), config);
|
|
|
+ pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
|
|
|
+ mockRatisProvider);
|
|
|
+
|
|
|
+
|
|
|
+ for (int i=0; i < pipelineCount; i++) {
|
|
|
+ pipelineManager.createPipeline(HddsProtos.ReplicationType.RATIS,
|
|
|
+ HddsProtos.ReplicationFactor.THREE);
|
|
|
+ }
|
|
|
+
|
|
|
+ for (ContainerInfo container : containers) {
|
|
|
+ container.setState(HddsProtos.LifeCycleState.CLOSED);
|
|
|
+ }
|
|
|
+
|
|
|
+ scmChillModeManager = new SCMChillModeManager(conf, containers,
|
|
|
+ pipelineManager, queue);
|
|
|
+
|
|
|
+ assertTrue(scmChillModeManager.getInChillMode());
|
|
|
+
|
|
|
+ testContainerThreshold(containers, 1.0);
|
|
|
+
|
|
|
+ List<Pipeline> pipelines = pipelineManager.getPipelines();
|
|
|
+
|
|
|
+ int healthyPipelineThresholdCount =
|
|
|
+ scmChillModeManager.getHealthyPipelineChillModeRule()
|
|
|
+ .getHealthyPipelineThresholdCount();
|
|
|
+ int oneReplicaThresholdCount =
|
|
|
+ scmChillModeManager.getOneReplicaPipelineChillModeRule()
|
|
|
+ .getThresholdCount();
|
|
|
+
|
|
|
+ // 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.
|
|
|
+ if (Math.max(healthyPipelinePercent, oneReplicaThresholdCount) == 0) {
|
|
|
+ firePipelineEvent(pipelines.get(0));
|
|
|
+ }
|
|
|
+
|
|
|
+ for (int i = 0; i < Math.max(healthyPipelineThresholdCount,
|
|
|
+ oneReplicaThresholdCount); i++) {
|
|
|
+ firePipelineEvent(pipelines.get(i));
|
|
|
+
|
|
|
+ if (i < healthyPipelineThresholdCount) {
|
|
|
+ checkHealthy(i + 1);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (i < oneReplicaThresholdCount) {
|
|
|
+ checkOpen(i + 1);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ return !scmChillModeManager.getInChillMode();
|
|
|
+ }, 100, 1000 * 5);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void checkHealthy(int expectedCount) throws Exception{
|
|
|
+ GenericTestUtils.waitFor(() -> scmChillModeManager
|
|
|
+ .getHealthyPipelineChillModeRule()
|
|
|
+ .getCurrentHealthyPipelineCount() == expectedCount,
|
|
|
+ 100, 5000);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void checkOpen(int expectedCount) throws Exception {
|
|
|
+ GenericTestUtils.waitFor(() -> scmChillModeManager
|
|
|
+ .getOneReplicaPipelineChillModeRule()
|
|
|
+ .getCurrentReportedPipelineCount() == expectedCount,
|
|
|
+ 1000, 5000);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void firePipelineEvent(Pipeline pipeline) throws Exception {
|
|
|
+ PipelineReportsProto.Builder reportBuilder =
|
|
|
+ PipelineReportsProto.newBuilder();
|
|
|
+
|
|
|
+ reportBuilder.addPipelineReport(PipelineReport.newBuilder()
|
|
|
+ .setPipelineID(pipeline.getId().getProtobuf()));
|
|
|
+ queue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT,
|
|
|
+ new PipelineReportFromDatanode(pipeline.getNodes().get(0),
|
|
|
+ reportBuilder.build()));
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
@Test
|
|
|
public void testDisableChillMode() {
|
|
|
OzoneConfiguration conf = new OzoneConfiguration(config);
|