|
@@ -23,6 +23,8 @@ import org.apache.hadoop.hdds.client.ReplicationType;
|
|
|
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.scm.container.ContainerID;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
|
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
@@ -33,6 +35,7 @@ import org.apache.hadoop.ozone.client.OzoneClient;
|
|
|
import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
|
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
|
|
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
|
|
|
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
|
|
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
|
|
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
|
@@ -45,6 +48,7 @@ import org.junit.Test;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.List;
|
|
|
+import java.util.Random;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
public class TestCloseContainerByPipeline {
|
|
@@ -88,6 +92,59 @@ public class TestCloseContainerByPipeline {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
|
|
|
+ OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
|
|
+ .createKey("testCloseContainer", 1024, ReplicationType.STAND_ALONE,
|
|
|
+ ReplicationFactor.ONE);
|
|
|
+ key.write("standalone".getBytes());
|
|
|
+ key.close();
|
|
|
+
|
|
|
+ //get the name of a valid container
|
|
|
+ KsmKeyArgs keyArgs =
|
|
|
+ new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
|
|
+ .setType(HddsProtos.ReplicationType.STAND_ALONE)
|
|
|
+ .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
|
|
+ .setKeyName("testCloseContainer").build();
|
|
|
+
|
|
|
+ KsmKeyLocationInfo ksmKeyLocationInfo =
|
|
|
+ cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
|
|
|
+ .get(0).getBlocksLatestVersionOnly().get(0);
|
|
|
+
|
|
|
+ long containerID = ksmKeyLocationInfo.getContainerID();
|
|
|
+ List<DatanodeDetails> datanodes =
|
|
|
+ cluster.getStorageContainerManager().getContainerInfo(containerID)
|
|
|
+ .getPipeline().getMachines();
|
|
|
+ Assert.assertTrue(datanodes.size() == 1);
|
|
|
+
|
|
|
+ DatanodeDetails datanodeDetails = datanodes.get(0);
|
|
|
+ HddsDatanodeService datanodeService = null;
|
|
|
+ Assert
|
|
|
+ .assertFalse(isContainerClosed(cluster, containerID, datanodeDetails));
|
|
|
+ for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) {
|
|
|
+ if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) {
|
|
|
+ datanodeService = datanodeServiceItr;
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ CommandHandler closeContainerHandler =
|
|
|
+ datanodeService.getDatanodeStateMachine().getCommandDispatcher()
|
|
|
+ .getCloseContainerHandler();
|
|
|
+ int lastInvocationCount = closeContainerHandler.getInvocationCount();
|
|
|
+ //send the order to close the container
|
|
|
+ cluster.getStorageContainerManager().getScmNodeManager()
|
|
|
+ .addDatanodeCommand(datanodeDetails.getUuid(),
|
|
|
+ new CloseContainerCommand(containerID,
|
|
|
+ HddsProtos.ReplicationType.STAND_ALONE));
|
|
|
+ GenericTestUtils
|
|
|
+ .waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails),
|
|
|
+ 500, 5 * 1000);
|
|
|
+ // Make sure the closeContainerCommandHandler is Invoked
|
|
|
+ Assert.assertTrue(
|
|
|
+ closeContainerHandler.getInvocationCount() > lastInvocationCount);
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testCloseContainerViaStandaAlone()
|
|
|
throws IOException, TimeoutException, InterruptedException {
|
|
@@ -205,13 +262,7 @@ public class TestCloseContainerByPipeline {
|
|
|
containerData =
|
|
|
datanodeService.getDatanodeStateMachine().getContainer()
|
|
|
.getContainerManager().readContainer(containerID);
|
|
|
- if (!containerData.isOpen()) {
|
|
|
- // make sure the closeContainerHandler on the Datanode is invoked
|
|
|
- Assert.assertTrue(
|
|
|
- datanodeService.getDatanodeStateMachine().getCommandDispatcher()
|
|
|
- .getCloseContainerHandler().getInvocationCount() > 0);
|
|
|
- return true;
|
|
|
- }
|
|
|
+ return !containerData.isOpen();
|
|
|
}
|
|
|
} catch (StorageContainerException e) {
|
|
|
throw new AssertionError(e);
|