|
@@ -17,15 +17,29 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdds.scm.chillmode;
|
|
|
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+
|
|
|
+import java.io.File;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
+import java.util.UUID;
|
|
|
+
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
|
|
|
import org.apache.hadoop.hdds.scm.HddsTestUtils;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
|
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
|
|
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.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;
|
|
@@ -33,9 +47,6 @@ import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.rules.Timeout;
|
|
|
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-import static org.junit.Assert.assertFalse;
|
|
|
-
|
|
|
/** Test class for SCMChillModeManager.
|
|
|
*/
|
|
|
public class TestSCMChillModeManager {
|
|
@@ -65,7 +76,7 @@ public class TestSCMChillModeManager {
|
|
|
|
|
|
@Test
|
|
|
public void testChillModeStateWithNullContainers() {
|
|
|
- new SCMChillModeManager(config, null, queue);
|
|
|
+ new SCMChillModeManager(config, null, null, queue);
|
|
|
}
|
|
|
|
|
|
private void testChillMode(int numContainers) throws Exception {
|
|
@@ -76,7 +87,8 @@ public class TestSCMChillModeManager {
|
|
|
for (ContainerInfo container : containers) {
|
|
|
container.setState(HddsProtos.LifeCycleState.OPEN);
|
|
|
}
|
|
|
- scmChillModeManager = new SCMChillModeManager(config, containers, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ config, containers, null, queue);
|
|
|
queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
|
scmChillModeManager);
|
|
|
assertTrue(scmChillModeManager.getInChillMode());
|
|
@@ -96,7 +108,8 @@ public class TestSCMChillModeManager {
|
|
|
for (ContainerInfo container : containers) {
|
|
|
container.setState(HddsProtos.LifeCycleState.CLOSED);
|
|
|
}
|
|
|
- scmChillModeManager = new SCMChillModeManager(config, containers, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ config, containers, null, queue);
|
|
|
queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
|
scmChillModeManager);
|
|
|
assertTrue(scmChillModeManager.getInChillMode());
|
|
@@ -118,7 +131,8 @@ public class TestSCMChillModeManager {
|
|
|
public void testDisableChillMode() {
|
|
|
OzoneConfiguration conf = new OzoneConfiguration(config);
|
|
|
conf.setBoolean(HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED, false);
|
|
|
- scmChillModeManager = new SCMChillModeManager(conf, containers, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ conf, containers, null, queue);
|
|
|
assertFalse(scmChillModeManager.getInChillMode());
|
|
|
}
|
|
|
|
|
@@ -149,7 +163,8 @@ public class TestSCMChillModeManager {
|
|
|
container.setState(HddsProtos.LifeCycleState.OPEN);
|
|
|
}
|
|
|
|
|
|
- scmChillModeManager = new SCMChillModeManager(config, containers, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ config, containers, null, queue);
|
|
|
queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
|
scmChillModeManager);
|
|
|
assertTrue(scmChillModeManager.getInChillMode());
|
|
@@ -173,7 +188,8 @@ public class TestSCMChillModeManager {
|
|
|
private void testChillModeDataNodes(int numOfDns) throws Exception {
|
|
|
OzoneConfiguration conf = new OzoneConfiguration(config);
|
|
|
conf.setInt(HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE, numOfDns);
|
|
|
- scmChillModeManager = new SCMChillModeManager(conf, containers, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ conf, containers, null, queue);
|
|
|
queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
|
scmChillModeManager);
|
|
|
// Assert SCM is in Chill mode.
|
|
@@ -212,4 +228,51 @@ public class TestSCMChillModeManager {
|
|
|
}, 100, 2000 * 9);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testChillModePipelineExitRule() throws Exception {
|
|
|
+ containers = new ArrayList<>();
|
|
|
+ containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
|
|
|
+ String storageDir = GenericTestUtils.getTempPath(
|
|
|
+ TestSCMChillModeManager.class.getName() + UUID.randomUUID());
|
|
|
+ try{
|
|
|
+ MockNodeManager nodeManager = new MockNodeManager(true, 1);
|
|
|
+ config.set(HddsConfigKeys.OZONE_METADATA_DIRS, storageDir);
|
|
|
+ // enable pipeline check
|
|
|
+ config.setBoolean(
|
|
|
+ HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
|
|
|
+
|
|
|
+ PipelineManager pipelineManager = new SCMPipelineManager(config,
|
|
|
+ nodeManager, queue);
|
|
|
+ scmChillModeManager = new SCMChillModeManager(
|
|
|
+ config, containers, pipelineManager, queue);
|
|
|
+ queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
|
+ scmChillModeManager);
|
|
|
+
|
|
|
+ queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
|
+ HddsTestUtils.createNodeRegistrationContainerReport(containers));
|
|
|
+ assertTrue(scmChillModeManager.getInChillMode());
|
|
|
+
|
|
|
+ // simulation a pipeline report to trigger the rule check
|
|
|
+ Pipeline pipeline = pipelineManager.createPipeline(
|
|
|
+ HddsProtos.ReplicationType.STAND_ALONE,
|
|
|
+ HddsProtos.ReplicationFactor.ONE);
|
|
|
+ PipelineReportsProto.Builder reportBuilder = PipelineReportsProto
|
|
|
+ .newBuilder();
|
|
|
+ reportBuilder.addPipelineReport(PipelineReport.newBuilder()
|
|
|
+ .setPipelineID(pipeline.getId().getProtobuf()));
|
|
|
+
|
|
|
+ queue.fireEvent(SCMEvents.PIPELINE_REPORT, new PipelineReportFromDatanode(
|
|
|
+ pipeline.getNodes().get(0), reportBuilder.build()));
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ return !scmChillModeManager.getInChillMode();
|
|
|
+ }, 100, 1000 * 10);
|
|
|
+ pipelineManager.close();
|
|
|
+ } finally {
|
|
|
+ config.setBoolean(
|
|
|
+ HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
|
|
|
+ false);
|
|
|
+ FileUtil.fullyDelete(new File(storageDir));
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|