|
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
|
+import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
@@ -43,7 +44,9 @@ import org.junit.Assert;
|
|
|
import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.UUID;
|
|
|
|
|
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
|
|
|
|
|
@@ -55,14 +58,22 @@ public class TestDeleteContainerHandler {
|
|
|
|
|
|
private static MiniOzoneCluster cluster;
|
|
|
private static OzoneConfiguration conf;
|
|
|
+ private static ObjectStore objectStore;
|
|
|
+ private static String volumeName = UUID.randomUUID().toString();
|
|
|
+ private static String bucketName = UUID.randomUUID().toString();
|
|
|
|
|
|
@BeforeClass
|
|
|
public static void setup() throws Exception {
|
|
|
conf = new OzoneConfiguration();
|
|
|
conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB");
|
|
|
cluster = MiniOzoneCluster.newBuilder(conf)
|
|
|
- .setNumDatanodes(3).build();
|
|
|
+ .setNumDatanodes(1).build();
|
|
|
cluster.waitForClusterToBeReady();
|
|
|
+
|
|
|
+ OzoneClient client = OzoneClientFactory.getClient(conf);
|
|
|
+ objectStore = client.getObjectStore();
|
|
|
+ objectStore.createVolume(volumeName);
|
|
|
+ objectStore.getVolume(volumeName).createBucket(bucketName);
|
|
|
}
|
|
|
|
|
|
@AfterClass
|
|
@@ -77,41 +88,27 @@ public class TestDeleteContainerHandler {
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 60000)
|
|
|
- public void testDeleteContainerRequestHandler() throws Exception {
|
|
|
+ public void testDeleteContainerRequestHandlerOnClosedContainer()
|
|
|
+ throws Exception {
|
|
|
|
|
|
//the easiest way to create an open container is creating a key
|
|
|
- OzoneClient client = OzoneClientFactory.getClient(conf);
|
|
|
- ObjectStore objectStore = client.getObjectStore();
|
|
|
- objectStore.createVolume("test");
|
|
|
- objectStore.getVolume("test").createBucket("test");
|
|
|
- OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
|
|
- .createKey("test", 1024, ReplicationType.RATIS,
|
|
|
- ReplicationFactor.THREE, new HashMap<>());
|
|
|
- key.write("test".getBytes());
|
|
|
- key.close();
|
|
|
|
|
|
- //get the name of a valid container
|
|
|
- OmKeyArgs keyArgs =
|
|
|
- new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
|
|
- .setType(HddsProtos.ReplicationType.RATIS)
|
|
|
- .setFactor(HddsProtos.ReplicationFactor.THREE).setDataSize(1024)
|
|
|
- .setKeyName("test").build();
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
|
|
|
- OmKeyLocationInfo omKeyLocationInfo =
|
|
|
- cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
|
|
- .get(0).getBlocksLatestVersionOnly().get(0);
|
|
|
+ // create key
|
|
|
+ createKey(keyName);
|
|
|
+
|
|
|
+ // get containerID of the key
|
|
|
+ ContainerID containerId = getContainerID(keyName);
|
|
|
|
|
|
- ContainerID containerId = ContainerID.valueof(
|
|
|
- omKeyLocationInfo.getContainerID());
|
|
|
ContainerInfo container = cluster.getStorageContainerManager()
|
|
|
.getContainerManager().getContainer(containerId);
|
|
|
+
|
|
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
|
|
.getPipelineManager().getPipeline(container.getPipelineID());
|
|
|
|
|
|
- // Take first node from the datanode list, and close the container (As we
|
|
|
- // have only three, the container will be created on three nodes)
|
|
|
// We need to close the container because delete container only happens
|
|
|
- // on closed containers.
|
|
|
+ // on closed containers with force flag set to false.
|
|
|
|
|
|
HddsDatanodeService hddsDatanodeService =
|
|
|
cluster.getHddsDatanodes().get(0);
|
|
@@ -121,9 +118,11 @@ public class TestDeleteContainerHandler {
|
|
|
|
|
|
DatanodeDetails datanodeDetails = hddsDatanodeService.getDatanodeDetails();
|
|
|
|
|
|
+ NodeManager nodeManager =
|
|
|
+ cluster.getStorageContainerManager().getScmNodeManager();
|
|
|
+
|
|
|
//send the order to close the container
|
|
|
- cluster.getStorageContainerManager().getScmNodeManager()
|
|
|
- .addDatanodeCommand(datanodeDetails.getUuid(),
|
|
|
+ nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
|
new CloseContainerCommand(containerId.getId(), pipeline.getId()));
|
|
|
|
|
|
GenericTestUtils.waitFor(() ->
|
|
@@ -138,44 +137,113 @@ public class TestDeleteContainerHandler {
|
|
|
Assert.assertFalse(isContainerDeleted(hddsDatanodeService,
|
|
|
containerId.getId()));
|
|
|
|
|
|
- // send delete container to one of the datanode
|
|
|
- cluster.getStorageContainerManager().getScmNodeManager()
|
|
|
- .addDatanodeCommand(datanodeDetails.getUuid(),
|
|
|
- new DeleteContainerCommand(containerId.getId()));
|
|
|
+ // send delete container to the datanode
|
|
|
+ nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
|
+ new DeleteContainerCommand(containerId.getId(), false));
|
|
|
|
|
|
GenericTestUtils.waitFor(() ->
|
|
|
isContainerDeleted(hddsDatanodeService, containerId.getId()),
|
|
|
500, 5 * 1000);
|
|
|
|
|
|
- // On another node, where container is open try to delete container
|
|
|
- HddsDatanodeService hddsDatanodeService1 =
|
|
|
- cluster.getHddsDatanodes().get(1);
|
|
|
- DatanodeDetails datanodeDetails1 =
|
|
|
- hddsDatanodeService1.getDatanodeDetails();
|
|
|
+ Assert.assertTrue(isContainerDeleted(hddsDatanodeService,
|
|
|
+ containerId.getId()));
|
|
|
|
|
|
- cluster.getStorageContainerManager().getScmNodeManager()
|
|
|
- .addDatanodeCommand(datanodeDetails1.getUuid(),
|
|
|
- new DeleteContainerCommand(containerId.getId()));
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDeleteContainerRequestHandlerOnOpenContainer()
|
|
|
+ throws Exception {
|
|
|
+
|
|
|
+ //the easiest way to create an open container is creating a key
|
|
|
+ String keyName = UUID.randomUUID().toString();
|
|
|
+
|
|
|
+ // create key
|
|
|
+ createKey(keyName);
|
|
|
+
|
|
|
+ // get containerID of the key
|
|
|
+ ContainerID containerId = getContainerID(keyName);
|
|
|
+
|
|
|
+ HddsDatanodeService hddsDatanodeService =
|
|
|
+ cluster.getHddsDatanodes().get(0);
|
|
|
+ DatanodeDetails datanodeDetails =
|
|
|
+ hddsDatanodeService.getDatanodeDetails();
|
|
|
+
|
|
|
+ NodeManager nodeManager =
|
|
|
+ cluster.getStorageContainerManager().getScmNodeManager();
|
|
|
+
|
|
|
+ // Send delete container command with force flag set to false.
|
|
|
+ nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
|
+ new DeleteContainerCommand(containerId.getId(), false));
|
|
|
|
|
|
// Here it should not delete it, and the container should exist in the
|
|
|
// containerset
|
|
|
-
|
|
|
int count = 1;
|
|
|
- // Checking for 10 seconds, whether it is containerSet, as after command
|
|
|
+ // Checking for 5 seconds, whether it is containerSet, as after command
|
|
|
// is issued, giving some time for it to process.
|
|
|
- while (!isContainerDeleted(hddsDatanodeService1, containerId.getId())) {
|
|
|
+ while (!isContainerDeleted(hddsDatanodeService, containerId.getId())) {
|
|
|
Thread.sleep(1000);
|
|
|
count++;
|
|
|
- if (count == 10) {
|
|
|
+ if (count == 5) {
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- Assert.assertFalse(isContainerDeleted(hddsDatanodeService1,
|
|
|
+ Assert.assertFalse(isContainerDeleted(hddsDatanodeService,
|
|
|
+ containerId.getId()));
|
|
|
+
|
|
|
+
|
|
|
+ // Now delete container with force flag set to true. now it should delete
|
|
|
+ // container
|
|
|
+
|
|
|
+ nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
|
|
|
+ new DeleteContainerCommand(containerId.getId(), true));
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() ->
|
|
|
+ isContainerDeleted(hddsDatanodeService, containerId.getId()),
|
|
|
+ 500, 5 * 1000);
|
|
|
+
|
|
|
+ Assert.assertTrue(isContainerDeleted(hddsDatanodeService,
|
|
|
containerId.getId()));
|
|
|
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * create a key with specified name.
|
|
|
+ * @param keyName
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private void createKey(String keyName) throws IOException {
|
|
|
+ OzoneOutputStream key = objectStore.getVolume(volumeName)
|
|
|
+ .getBucket(bucketName)
|
|
|
+ .createKey(keyName, 1024, ReplicationType.STAND_ALONE,
|
|
|
+ ReplicationFactor.ONE, new HashMap<>());
|
|
|
+ key.write("test".getBytes());
|
|
|
+ key.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Return containerID of the key.
|
|
|
+ * @param keyName
|
|
|
+ * @return ContainerID
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private ContainerID getContainerID(String keyName) throws IOException {
|
|
|
+ OmKeyArgs keyArgs =
|
|
|
+ new OmKeyArgs.Builder().setVolumeName(volumeName)
|
|
|
+ .setBucketName(bucketName)
|
|
|
+ .setType(HddsProtos.ReplicationType.STAND_ALONE)
|
|
|
+ .setFactor(HddsProtos.ReplicationFactor.ONE)
|
|
|
+ .setKeyName(keyName).build();
|
|
|
+
|
|
|
+ OmKeyLocationInfo omKeyLocationInfo =
|
|
|
+ cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
|
|
+ .get(0).getBlocksLatestVersionOnly().get(0);
|
|
|
+
|
|
|
+ return ContainerID.valueof(
|
|
|
+ omKeyLocationInfo.getContainerID());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Checks whether is closed or not on a datanode.
|
|
|
* @param hddsDatanodeService
|