|
@@ -17,22 +17,32 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdds.scm.pipeline;
|
|
|
|
|
|
+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.protocol.proto.HddsProtos;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
|
|
|
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
+import org.apache.hadoop.hdds.scm.TestUtils;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
|
|
|
+import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineActionsFromDatanode;
|
|
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
|
|
+import org.apache.hadoop.hdds.server.events.EventQueue;
|
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
|
-import org.junit.AfterClass;
|
|
|
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
|
|
+import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
-import org.junit.BeforeClass;
|
|
|
+import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.List;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
|
|
@@ -43,35 +53,36 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.R
|
|
|
*/
|
|
|
public class TestPipelineClose {
|
|
|
|
|
|
- private static MiniOzoneCluster cluster;
|
|
|
- private static OzoneConfiguration conf;
|
|
|
- private static StorageContainerManager scm;
|
|
|
- private static ContainerWithPipeline ratisContainer1;
|
|
|
- private static ContainerWithPipeline ratisContainer2;
|
|
|
- private static ContainerManager containerManager;
|
|
|
- private static PipelineManager pipelineManager;
|
|
|
+ private MiniOzoneCluster cluster;
|
|
|
+ private OzoneConfiguration conf;
|
|
|
+ private StorageContainerManager scm;
|
|
|
+ private ContainerWithPipeline ratisContainer;
|
|
|
+ private ContainerManager containerManager;
|
|
|
+ private PipelineManager pipelineManager;
|
|
|
|
|
|
+ private long pipelineDestroyTimeoutInMillis;
|
|
|
/**
|
|
|
* Create a MiniDFSCluster for testing.
|
|
|
*
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- @BeforeClass
|
|
|
- public static void init() throws Exception {
|
|
|
+ @Before
|
|
|
+ public void init() throws Exception {
|
|
|
conf = new OzoneConfiguration();
|
|
|
- cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(6).build();
|
|
|
+ cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
|
|
|
+ conf.setTimeDuration(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, 1000,
|
|
|
+ TimeUnit.MILLISECONDS);
|
|
|
+ pipelineDestroyTimeoutInMillis = 5000;
|
|
|
+ conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT,
|
|
|
+ pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS);
|
|
|
cluster.waitForClusterToBeReady();
|
|
|
scm = cluster.getStorageContainerManager();
|
|
|
containerManager = scm.getContainerManager();
|
|
|
pipelineManager = scm.getPipelineManager();
|
|
|
- ContainerInfo containerInfo1 = containerManager
|
|
|
+ ContainerInfo containerInfo = containerManager
|
|
|
.allocateContainer(RATIS, THREE, "testOwner");
|
|
|
- ratisContainer1 = new ContainerWithPipeline(containerInfo1,
|
|
|
- pipelineManager.getPipeline(containerInfo1.getPipelineID()));
|
|
|
- ContainerInfo containerInfo2 = containerManager
|
|
|
- .allocateContainer(RATIS, THREE, "testOwner");
|
|
|
- ratisContainer2 = new ContainerWithPipeline(containerInfo2,
|
|
|
- pipelineManager.getPipeline(containerInfo2.getPipelineID()));
|
|
|
+ ratisContainer = new ContainerWithPipeline(containerInfo,
|
|
|
+ pipelineManager.getPipeline(containerInfo.getPipelineID()));
|
|
|
pipelineManager = scm.getPipelineManager();
|
|
|
// At this stage, there should be 2 pipeline one with 1 open container each.
|
|
|
// Try closing the both the pipelines, one with a closed container and
|
|
@@ -81,8 +92,8 @@ public class TestPipelineClose {
|
|
|
/**
|
|
|
* Shutdown MiniDFSCluster.
|
|
|
*/
|
|
|
- @AfterClass
|
|
|
- public static void shutdown() {
|
|
|
+ @After
|
|
|
+ public void shutdown() {
|
|
|
if (cluster != null) {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
@@ -91,9 +102,9 @@ public class TestPipelineClose {
|
|
|
@Test
|
|
|
public void testPipelineCloseWithClosedContainer() throws IOException {
|
|
|
Set<ContainerID> set = pipelineManager
|
|
|
- .getContainersInPipeline(ratisContainer1.getPipeline().getId());
|
|
|
+ .getContainersInPipeline(ratisContainer.getPipeline().getId());
|
|
|
|
|
|
- ContainerID cId = ratisContainer1.getContainerInfo().containerID();
|
|
|
+ ContainerID cId = ratisContainer.getContainerInfo().containerID();
|
|
|
Assert.assertEquals(1, set.size());
|
|
|
set.forEach(containerID -> Assert.assertEquals(containerID, cId));
|
|
|
|
|
@@ -105,16 +116,16 @@ public class TestPipelineClose {
|
|
|
.updateContainerState(cId, HddsProtos.LifeCycleEvent.CLOSE);
|
|
|
|
|
|
Set<ContainerID> setClosed = pipelineManager
|
|
|
- .getContainersInPipeline(ratisContainer1.getPipeline().getId());
|
|
|
+ .getContainersInPipeline(ratisContainer.getPipeline().getId());
|
|
|
Assert.assertEquals(0, setClosed.size());
|
|
|
|
|
|
- pipelineManager.finalizePipeline(ratisContainer1.getPipeline().getId());
|
|
|
+ pipelineManager.finalizePipeline(ratisContainer.getPipeline().getId());
|
|
|
Pipeline pipeline1 = pipelineManager
|
|
|
- .getPipeline(ratisContainer1.getPipeline().getId());
|
|
|
+ .getPipeline(ratisContainer.getPipeline().getId());
|
|
|
Assert.assertEquals(Pipeline.PipelineState.CLOSED,
|
|
|
pipeline1.getPipelineState());
|
|
|
pipelineManager.removePipeline(pipeline1.getId());
|
|
|
- for (DatanodeDetails dn : ratisContainer1.getPipeline().getNodes()) {
|
|
|
+ for (DatanodeDetails dn : ratisContainer.getPipeline().getNodes()) {
|
|
|
// Assert that the pipeline has been removed from Node2PipelineMap as well
|
|
|
Assert.assertEquals(scm.getScmNodeManager().getPipelines(
|
|
|
dn).size(), 0);
|
|
@@ -125,17 +136,80 @@ public class TestPipelineClose {
|
|
|
public void testPipelineCloseWithOpenContainer() throws IOException,
|
|
|
TimeoutException, InterruptedException {
|
|
|
Set<ContainerID> setOpen = pipelineManager.getContainersInPipeline(
|
|
|
- ratisContainer2.getPipeline().getId());
|
|
|
+ ratisContainer.getPipeline().getId());
|
|
|
Assert.assertEquals(1, setOpen.size());
|
|
|
|
|
|
- ContainerID cId2 = ratisContainer2.getContainerInfo().containerID();
|
|
|
- pipelineManager.finalizePipeline(ratisContainer2.getPipeline().getId());
|
|
|
+ ContainerID cId2 = ratisContainer.getContainerInfo().containerID();
|
|
|
+ pipelineManager.finalizePipeline(ratisContainer.getPipeline().getId());
|
|
|
Assert.assertEquals(Pipeline.PipelineState.CLOSED,
|
|
|
pipelineManager.getPipeline(
|
|
|
- ratisContainer2.getPipeline().getId()).getPipelineState());
|
|
|
+ ratisContainer.getPipeline().getId()).getPipelineState());
|
|
|
Pipeline pipeline2 = pipelineManager
|
|
|
- .getPipeline(ratisContainer2.getPipeline().getId());
|
|
|
+ .getPipeline(ratisContainer.getPipeline().getId());
|
|
|
Assert.assertEquals(Pipeline.PipelineState.CLOSED,
|
|
|
pipeline2.getPipelineState());
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testPipelineCloseWithPipelineAction() throws Exception {
|
|
|
+ List<DatanodeDetails> dns = ratisContainer.getPipeline().getNodes();
|
|
|
+ PipelineActionsFromDatanode
|
|
|
+ pipelineActionsFromDatanode = TestUtils
|
|
|
+ .getPipelineActionFromDatanode(dns.get(0),
|
|
|
+ ratisContainer.getPipeline().getId());
|
|
|
+ // send closing action for pipeline
|
|
|
+ PipelineActionHandler pipelineActionHandler =
|
|
|
+ new PipelineActionHandler(pipelineManager, conf);
|
|
|
+ pipelineActionHandler
|
|
|
+ .onMessage(pipelineActionsFromDatanode, new EventQueue());
|
|
|
+ Thread.sleep((int) (pipelineDestroyTimeoutInMillis * 1.2));
|
|
|
+ OzoneContainer ozoneContainer =
|
|
|
+ cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
|
|
|
+ .getContainer();
|
|
|
+ List<PipelineReport> pipelineReports =
|
|
|
+ ozoneContainer.getPipelineReport().getPipelineReportList();
|
|
|
+ for (PipelineReport pipelineReport : pipelineReports) {
|
|
|
+ // ensure the pipeline is not reported by any dn
|
|
|
+ Assert.assertNotEquals(
|
|
|
+ PipelineID.getFromProtobuf(pipelineReport.getPipelineID()),
|
|
|
+ ratisContainer.getPipeline().getId());
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ pipelineManager.getPipeline(ratisContainer.getPipeline().getId());
|
|
|
+ Assert.fail("Pipeline should not exist in SCM");
|
|
|
+ } catch (PipelineNotFoundException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testPipelineCloseWithPipelineReport() throws IOException {
|
|
|
+ Pipeline pipeline = ratisContainer.getPipeline();
|
|
|
+ pipelineManager.finalizePipeline(pipeline.getId());
|
|
|
+ // remove pipeline from SCM
|
|
|
+ pipelineManager.removePipeline(pipeline.getId());
|
|
|
+
|
|
|
+ for (DatanodeDetails dn : pipeline.getNodes()) {
|
|
|
+ PipelineReportFromDatanode pipelineReport =
|
|
|
+ TestUtils.getPipelineReportFromDatanode(dn, pipeline.getId());
|
|
|
+ PipelineReportHandler pipelineReportHandler =
|
|
|
+ new PipelineReportHandler(pipelineManager, conf);
|
|
|
+ // on receiving pipeline report for the pipeline, pipeline report handler
|
|
|
+ // should destroy the pipeline for the dn
|
|
|
+ pipelineReportHandler.onMessage(pipelineReport, new EventQueue());
|
|
|
+ }
|
|
|
+
|
|
|
+ OzoneContainer ozoneContainer =
|
|
|
+ cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
|
|
|
+ .getContainer();
|
|
|
+ List<PipelineReport> pipelineReports =
|
|
|
+ ozoneContainer.getPipelineReport().getPipelineReportList();
|
|
|
+ for (PipelineReport pipelineReport : pipelineReports) {
|
|
|
+ // pipeline should not be reported by any dn
|
|
|
+ Assert.assertNotEquals(
|
|
|
+ PipelineID.getFromProtobuf(pipelineReport.getPipelineID()),
|
|
|
+ ratisContainer.getPipeline().getId());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|