|
@@ -22,9 +22,11 @@ import org.apache.hadoop.hdds.client.ReplicationType;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
|
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
import org.apache.hadoop.ozone.client.ObjectStore;
|
|
import org.apache.hadoop.ozone.client.ObjectStore;
|
|
@@ -32,7 +34,6 @@ import org.apache.hadoop.ozone.client.OzoneClient;
|
|
import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
|
import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
|
|
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
|
|
-import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
|
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
|
@@ -63,8 +64,9 @@ public class TestCloseContainerByPipeline {
|
|
@BeforeClass
|
|
@BeforeClass
|
|
public static void init() throws Exception {
|
|
public static void init() throws Exception {
|
|
conf = new OzoneConfiguration();
|
|
conf = new OzoneConfiguration();
|
|
|
|
+ conf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, "1");
|
|
cluster = MiniOzoneCluster.newBuilder(conf)
|
|
cluster = MiniOzoneCluster.newBuilder(conf)
|
|
- .setNumDatanodes(3).build();
|
|
|
|
|
|
+ .setNumDatanodes(9).build();
|
|
cluster.waitForClusterToBeReady();
|
|
cluster.waitForClusterToBeReady();
|
|
//the easiest way to create an open container is creating a key
|
|
//the easiest way to create an open container is creating a key
|
|
client = OzoneClientFactory.getClient(conf);
|
|
client = OzoneClientFactory.getClient(conf);
|
|
@@ -86,7 +88,7 @@ public class TestCloseContainerByPipeline {
|
|
@Test
|
|
@Test
|
|
public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
|
|
public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
|
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
|
- .createKey("testCloseContainer", 1024, ReplicationType.STAND_ALONE,
|
|
|
|
|
|
+ .createKey("standalone", 1024, ReplicationType.RATIS,
|
|
ReplicationFactor.ONE);
|
|
ReplicationFactor.ONE);
|
|
key.write("standalone".getBytes());
|
|
key.write("standalone".getBytes());
|
|
key.close();
|
|
key.close();
|
|
@@ -94,10 +96,9 @@ public class TestCloseContainerByPipeline {
|
|
//get the name of a valid container
|
|
//get the name of a valid container
|
|
OmKeyArgs keyArgs =
|
|
OmKeyArgs keyArgs =
|
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
|
- .setType(HddsProtos.ReplicationType.STAND_ALONE)
|
|
|
|
|
|
+ .setType(HddsProtos.ReplicationType.RATIS)
|
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
|
- .setKeyName("testCloseContainer").build();
|
|
|
|
-
|
|
|
|
|
|
+ .setKeyName("standalone").build();
|
|
OmKeyLocationInfo omKeyLocationInfo =
|
|
OmKeyLocationInfo omKeyLocationInfo =
|
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
|
@@ -127,8 +128,7 @@ public class TestCloseContainerByPipeline {
|
|
//send the order to close the container
|
|
//send the order to close the container
|
|
cluster.getStorageContainerManager().getScmNodeManager()
|
|
cluster.getStorageContainerManager().getScmNodeManager()
|
|
.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
- new CloseContainerCommand(containerID,
|
|
|
|
- HddsProtos.ReplicationType.STAND_ALONE, pipeline.getId()));
|
|
|
|
|
|
+ new CloseContainerCommand(containerID, pipeline.getId()));
|
|
GenericTestUtils
|
|
GenericTestUtils
|
|
.waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails),
|
|
.waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails),
|
|
500, 5 * 1000);
|
|
500, 5 * 1000);
|
|
@@ -142,7 +142,7 @@ public class TestCloseContainerByPipeline {
|
|
throws IOException, TimeoutException, InterruptedException {
|
|
throws IOException, TimeoutException, InterruptedException {
|
|
|
|
|
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
|
- .createKey("standalone", 1024, ReplicationType.STAND_ALONE,
|
|
|
|
|
|
+ .createKey("standalone", 1024, ReplicationType.RATIS,
|
|
ReplicationFactor.ONE);
|
|
ReplicationFactor.ONE);
|
|
key.write("standalone".getBytes());
|
|
key.write("standalone".getBytes());
|
|
key.close();
|
|
key.close();
|
|
@@ -150,7 +150,7 @@ public class TestCloseContainerByPipeline {
|
|
//get the name of a valid container
|
|
//get the name of a valid container
|
|
OmKeyArgs keyArgs =
|
|
OmKeyArgs keyArgs =
|
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
|
- .setType(HddsProtos.ReplicationType.STAND_ALONE)
|
|
|
|
|
|
+ .setType(HddsProtos.ReplicationType.RATIS)
|
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
|
.setKeyName("standalone").build();
|
|
.setKeyName("standalone").build();
|
|
|
|
|
|
@@ -170,30 +170,20 @@ public class TestCloseContainerByPipeline {
|
|
Assert
|
|
Assert
|
|
.assertFalse(isContainerClosed(cluster, containerID, datanodeDetails));
|
|
.assertFalse(isContainerClosed(cluster, containerID, datanodeDetails));
|
|
|
|
|
|
- GenericTestUtils.LogCapturer logCapturer =
|
|
|
|
- GenericTestUtils.LogCapturer.captureLogs(OzoneContainer.LOG);
|
|
|
|
- //send the order to close the container
|
|
|
|
|
|
+ // Send the order to close the container, give random pipeline id so that
|
|
|
|
+ // the container will not be closed via RATIS
|
|
cluster.getStorageContainerManager().getScmNodeManager()
|
|
cluster.getStorageContainerManager().getScmNodeManager()
|
|
.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
- new CloseContainerCommand(containerID,
|
|
|
|
- HddsProtos.ReplicationType.STAND_ALONE, pipeline.getId()));
|
|
|
|
-
|
|
|
|
- // The log will appear after the state changed to closed in standalone,
|
|
|
|
- // wait for the log to ensure the operation has been done.
|
|
|
|
- GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains(
|
|
|
|
- "submitting CloseContainer request over STAND_ALONE server for"
|
|
|
|
- + " container " + containerID), 500, 5 * 1000);
|
|
|
|
|
|
+ new CloseContainerCommand(containerID, PipelineID.randomId()));
|
|
|
|
|
|
//double check if it's really closed (waitFor also throws an exception)
|
|
//double check if it's really closed (waitFor also throws an exception)
|
|
- Assert.assertTrue(isContainerClosed(cluster, containerID, datanodeDetails));
|
|
|
|
- Assert.assertTrue(logCapturer.getOutput().contains(
|
|
|
|
- "submitting CloseContainer request over STAND_ALONE server for"
|
|
|
|
- + " container " + containerID));
|
|
|
|
- // Make sure it was really closed via StandAlone not Ratis server
|
|
|
|
- Assert.assertFalse((logCapturer.getOutput().contains(
|
|
|
|
- "submitting CloseContainer request over RATIS server for container "
|
|
|
|
- + containerID)));
|
|
|
|
- logCapturer.stopCapturing();
|
|
|
|
|
|
+ // TODO: change the below line after implementing QUASI_CLOSED to CLOSED
|
|
|
|
+ // logic. The container will be QUASI closed as of now
|
|
|
|
+ GenericTestUtils
|
|
|
|
+ .waitFor(() -> isContainerQuasiClosed(
|
|
|
|
+ cluster, containerID, datanodeDetails), 500, 5 * 1000);
|
|
|
|
+ Assert.assertTrue(
|
|
|
|
+ isContainerQuasiClosed(cluster, containerID, datanodeDetails));
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -224,18 +214,14 @@ public class TestCloseContainerByPipeline {
|
|
List<DatanodeDetails> datanodes = pipeline.getNodes();
|
|
List<DatanodeDetails> datanodes = pipeline.getNodes();
|
|
Assert.assertEquals(3, datanodes.size());
|
|
Assert.assertEquals(3, datanodes.size());
|
|
|
|
|
|
- GenericTestUtils.LogCapturer logCapturer =
|
|
|
|
- GenericTestUtils.LogCapturer.captureLogs(OzoneContainer.LOG);
|
|
|
|
-
|
|
|
|
for (DatanodeDetails details : datanodes) {
|
|
for (DatanodeDetails details : datanodes) {
|
|
Assert.assertFalse(isContainerClosed(cluster, containerID, details));
|
|
Assert.assertFalse(isContainerClosed(cluster, containerID, details));
|
|
//send the order to close the container
|
|
//send the order to close the container
|
|
cluster.getStorageContainerManager().getScmNodeManager()
|
|
cluster.getStorageContainerManager().getScmNodeManager()
|
|
.addDatanodeCommand(details.getUuid(),
|
|
.addDatanodeCommand(details.getUuid(),
|
|
- new CloseContainerCommand(containerID,
|
|
|
|
- HddsProtos.ReplicationType.RATIS, pipeline.getId()));
|
|
|
|
|
|
+ new CloseContainerCommand(containerID, pipeline.getId()));
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+ // Make sure that it is CLOSED
|
|
for (DatanodeDetails datanodeDetails : datanodes) {
|
|
for (DatanodeDetails datanodeDetails : datanodes) {
|
|
GenericTestUtils.waitFor(
|
|
GenericTestUtils.waitFor(
|
|
() -> isContainerClosed(cluster, containerID, datanodeDetails), 500,
|
|
() -> isContainerClosed(cluster, containerID, datanodeDetails), 500,
|
|
@@ -244,14 +230,6 @@ public class TestCloseContainerByPipeline {
|
|
Assert.assertTrue(isContainerClosed(cluster,
|
|
Assert.assertTrue(isContainerClosed(cluster,
|
|
containerID, datanodeDetails));
|
|
containerID, datanodeDetails));
|
|
}
|
|
}
|
|
- // Make sure it was really closed via Ratis not STAND_ALONE server
|
|
|
|
- Assert.assertFalse(logCapturer.getOutput().contains(
|
|
|
|
- "submitting CloseContainer request over STAND_ALONE "
|
|
|
|
- + "server for container " + containerID));
|
|
|
|
- Assert.assertTrue((logCapturer.getOutput().contains(
|
|
|
|
- "submitting CloseContainer request over RATIS server for container "
|
|
|
|
- + containerID)));
|
|
|
|
- logCapturer.stopCapturing();
|
|
|
|
}
|
|
}
|
|
|
|
|
|
private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
|
|
private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
|
|
@@ -267,4 +245,18 @@ public class TestCloseContainerByPipeline {
|
|
}
|
|
}
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private Boolean isContainerQuasiClosed(MiniOzoneCluster miniCluster,
|
|
|
|
+ long containerID, DatanodeDetails datanode) {
|
|
|
|
+ ContainerData containerData;
|
|
|
|
+ for (HddsDatanodeService datanodeService : miniCluster.getHddsDatanodes()) {
|
|
|
|
+ if (datanode.equals(datanodeService.getDatanodeDetails())) {
|
|
|
|
+ containerData =
|
|
|
|
+ datanodeService.getDatanodeStateMachine().getContainer()
|
|
|
|
+ .getContainerSet().getContainer(containerID).getContainerData();
|
|
|
|
+ return containerData.isQuasiClosed();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
}
|
|
}
|