|
@@ -16,306 +16,187 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
|
|
|
|
|
|
-import org.apache.commons.io.FileUtils;
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
-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.datanode.proto.ContainerProtos;
|
|
|
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
-import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
|
|
-import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
|
|
+import org.apache.hadoop.ozone.container.common.interfaces.Handler;
|
|
|
import org.apache.hadoop.ozone.container.common.statemachine
|
|
|
.DatanodeStateMachine;
|
|
|
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
|
|
|
+import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
|
|
+import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
|
|
|
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
|
|
-import org.apache.hadoop.test.GenericTestUtils;
|
|
|
-import org.apache.hadoop.hdds.ratis.RatisHelper;
|
|
|
-import org.apache.ratis.client.RaftClient;
|
|
|
-import org.apache.ratis.protocol.RaftGroup;
|
|
|
-import org.apache.ratis.protocol.RaftGroupId;
|
|
|
-import org.apache.ratis.protocol.RaftPeer;
|
|
|
-import org.apache.ratis.retry.RetryPolicy;
|
|
|
-import org.apache.ratis.rpc.SupportedRpcType;
|
|
|
-import org.apache.ratis.util.TimeDuration;
|
|
|
-import org.junit.AfterClass;
|
|
|
-import org.junit.Assert;
|
|
|
+import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
-import org.mockito.Mockito;
|
|
|
|
|
|
-import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.Random;
|
|
|
import java.util.UUID;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
+
|
|
|
+import static java.util.Collections.singletonMap;
|
|
|
+import static org.apache.hadoop.ozone.OzoneConsts.GB;
|
|
|
+import static org.mockito.ArgumentMatchers.any;
|
|
|
+import static org.mockito.ArgumentMatchers.eq;
|
|
|
+import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.never;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
|
|
|
/**
|
|
|
* Test cases to verify CloseContainerCommandHandler in datanode.
|
|
|
*/
|
|
|
public class TestCloseContainerCommandHandler {
|
|
|
|
|
|
- private final StateContext context = Mockito.mock(StateContext.class);
|
|
|
- private final Random random = new Random();
|
|
|
- private static File testDir;
|
|
|
+ private static final long CONTAINER_ID = 123L;
|
|
|
+
|
|
|
+ private OzoneContainer ozoneContainer;
|
|
|
+ private StateContext context;
|
|
|
+ private XceiverServerSpi writeChannel;
|
|
|
+ private Container container;
|
|
|
+ private Handler containerHandler;
|
|
|
+ private PipelineID pipelineID;
|
|
|
+ private PipelineID nonExistentPipelineID = PipelineID.randomId();
|
|
|
+
|
|
|
+ private CloseContainerCommandHandler subject =
|
|
|
+ new CloseContainerCommandHandler();
|
|
|
+
|
|
|
+ @Before
|
|
|
+ public void before() throws Exception {
|
|
|
+ context = mock(StateContext.class);
|
|
|
+ DatanodeStateMachine dnStateMachine = mock(DatanodeStateMachine.class);
|
|
|
+ when(dnStateMachine.getDatanodeDetails())
|
|
|
+ .thenReturn(randomDatanodeDetails());
|
|
|
+ when(context.getParent()).thenReturn(dnStateMachine);
|
|
|
+
|
|
|
+ pipelineID = PipelineID.randomId();
|
|
|
+
|
|
|
+ KeyValueContainerData data = new KeyValueContainerData(CONTAINER_ID, GB,
|
|
|
+ pipelineID.getId().toString(), null);
|
|
|
+
|
|
|
+ container = new KeyValueContainer(data, new OzoneConfiguration());
|
|
|
+ ContainerSet containerSet = new ContainerSet();
|
|
|
+ containerSet.addContainer(container);
|
|
|
+
|
|
|
+ containerHandler = mock(Handler.class);
|
|
|
+ ContainerController controller = new ContainerController(containerSet,
|
|
|
+ singletonMap(ContainerProtos.ContainerType.KeyValueContainer,
|
|
|
+ containerHandler));
|
|
|
+
|
|
|
+ writeChannel = mock(XceiverServerSpi.class);
|
|
|
+ ozoneContainer = mock(OzoneContainer.class);
|
|
|
+ when(ozoneContainer.getController()).thenReturn(controller);
|
|
|
+ when(ozoneContainer.getContainerSet()).thenReturn(containerSet);
|
|
|
+ when(ozoneContainer.getWriteChannel()).thenReturn(writeChannel);
|
|
|
+ when(writeChannel.isExist(pipelineID.getProtobuf())).thenReturn(true);
|
|
|
+ when(writeChannel.isExist(nonExistentPipelineID.getProtobuf()))
|
|
|
+ .thenReturn(false);
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
|
- public void testCloseContainerViaRatis()
|
|
|
- throws Exception {
|
|
|
- final OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
- final DatanodeDetails datanodeDetails = randomDatanodeDetails();
|
|
|
- final OzoneContainer ozoneContainer =
|
|
|
- getOzoneContainer(conf, datanodeDetails);
|
|
|
- ozoneContainer.start(UUID.randomUUID().toString());
|
|
|
- try {
|
|
|
- final Container container =
|
|
|
- createContainer(conf, datanodeDetails, ozoneContainer);
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(1)).triggerHeartbeat();
|
|
|
- final long containerId = container.getContainerData().getContainerID();
|
|
|
- final PipelineID pipelineId = PipelineID.valueOf(UUID.fromString(
|
|
|
- container.getContainerData().getOriginPipelineId()));
|
|
|
-
|
|
|
- // We have created a container via ratis.
|
|
|
- // Now close the container on ratis.
|
|
|
- final CloseContainerCommandHandler closeHandler =
|
|
|
- new CloseContainerCommandHandler();
|
|
|
- final CloseContainerCommand command = new CloseContainerCommand(
|
|
|
- containerId, pipelineId);
|
|
|
-
|
|
|
- closeHandler.handle(command, ozoneContainer, context, null);
|
|
|
-
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
|
|
|
- ozoneContainer.getContainerSet().getContainer(containerId)
|
|
|
- .getContainerState());
|
|
|
-
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(3)).triggerHeartbeat();
|
|
|
- } finally {
|
|
|
- ozoneContainer.stop();
|
|
|
- }
|
|
|
+ public void closeContainerWithPipeline() throws Exception {
|
|
|
+ // close a container that's associated with an existing pipeline
|
|
|
+ subject.handle(closeWithKnownPipeline(), ozoneContainer, context, null);
|
|
|
+
|
|
|
+ verify(containerHandler)
|
|
|
+ .markContainerForClose(container);
|
|
|
+ verify(writeChannel)
|
|
|
+ .submitRequest(any(), eq(pipelineID.getProtobuf()));
|
|
|
+ verify(containerHandler, never())
|
|
|
+ .quasiCloseContainer(container);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testCloseContainerViaStandalone()
|
|
|
- throws Exception {
|
|
|
- final OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
- final DatanodeDetails datanodeDetails = randomDatanodeDetails();
|
|
|
- final OzoneContainer ozoneContainer =
|
|
|
- getOzoneContainer(conf, datanodeDetails);
|
|
|
- ozoneContainer.start(UUID.randomUUID().toString());
|
|
|
- try {
|
|
|
- final Container container =
|
|
|
- createContainer(conf, datanodeDetails, ozoneContainer);
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(1)).triggerHeartbeat();
|
|
|
- final long containerId = container.getContainerData().getContainerID();
|
|
|
- // To quasi close specify a pipeline which doesn't exist in the datanode.
|
|
|
- final PipelineID pipelineId = PipelineID.randomId();
|
|
|
-
|
|
|
- // We have created a container via ratis. Now quasi close it.
|
|
|
- final CloseContainerCommandHandler closeHandler =
|
|
|
- new CloseContainerCommandHandler();
|
|
|
- final CloseContainerCommand command = new CloseContainerCommand(
|
|
|
- containerId, pipelineId);
|
|
|
-
|
|
|
- closeHandler.handle(command, ozoneContainer, context, null);
|
|
|
-
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.QUASI_CLOSED,
|
|
|
- ozoneContainer.getContainerSet().getContainer(containerId)
|
|
|
- .getContainerState());
|
|
|
-
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(3)).triggerHeartbeat();
|
|
|
- } finally {
|
|
|
- ozoneContainer.stop();
|
|
|
- }
|
|
|
+ public void closeContainerWithoutPipeline() throws IOException {
|
|
|
+ // close a container that's NOT associated with an open pipeline
|
|
|
+ subject.handle(closeWithUnknownPipeline(), ozoneContainer, context, null);
|
|
|
+
|
|
|
+ verify(containerHandler)
|
|
|
+ .markContainerForClose(container);
|
|
|
+ verify(writeChannel, never())
|
|
|
+ .submitRequest(any(), any());
|
|
|
+ verify(containerHandler)
|
|
|
+ .quasiCloseContainer(container);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void forceCloseQuasiClosedContainer() throws Exception {
|
|
|
+ // force-close a container that's already quasi closed
|
|
|
+ container.getContainerData()
|
|
|
+ .setState(ContainerProtos.ContainerDataProto.State.QUASI_CLOSED);
|
|
|
+
|
|
|
+ subject.handle(forceCloseWithoutPipeline(), ozoneContainer, context, null);
|
|
|
+
|
|
|
+ verify(writeChannel, never())
|
|
|
+ .submitRequest(any(), any());
|
|
|
+ verify(containerHandler)
|
|
|
+ .closeContainer(container);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testQuasiCloseToClose() throws Exception {
|
|
|
- final OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
- final DatanodeDetails datanodeDetails = randomDatanodeDetails();
|
|
|
- final OzoneContainer ozoneContainer =
|
|
|
- getOzoneContainer(conf, datanodeDetails);
|
|
|
- ozoneContainer.start(UUID.randomUUID().toString());
|
|
|
- try {
|
|
|
- final Container container =
|
|
|
- createContainer(conf, datanodeDetails, ozoneContainer);
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(1)).triggerHeartbeat();
|
|
|
- final long containerId = container.getContainerData().getContainerID();
|
|
|
- // A pipeline which doesn't exist in the datanode.
|
|
|
- final PipelineID pipelineId = PipelineID.randomId();
|
|
|
-
|
|
|
- // We have created a container via ratis. Now quasi close it.
|
|
|
- final CloseContainerCommandHandler closeHandler =
|
|
|
- new CloseContainerCommandHandler();
|
|
|
- final CloseContainerCommand command = new CloseContainerCommand(
|
|
|
- containerId, pipelineId);
|
|
|
-
|
|
|
- closeHandler.handle(command, ozoneContainer, context, null);
|
|
|
-
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.QUASI_CLOSED,
|
|
|
- ozoneContainer.getContainerSet().getContainer(containerId)
|
|
|
- .getContainerState());
|
|
|
-
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(3)).triggerHeartbeat();
|
|
|
-
|
|
|
- // The container is quasi closed. Force close the container now.
|
|
|
- final CloseContainerCommand closeCommand = new CloseContainerCommand(
|
|
|
- containerId, pipelineId, true);
|
|
|
-
|
|
|
- closeHandler.handle(closeCommand, ozoneContainer, context, null);
|
|
|
-
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
|
|
|
- ozoneContainer.getContainerSet().getContainer(containerId)
|
|
|
- .getContainerState());
|
|
|
-
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(4)).triggerHeartbeat();
|
|
|
- } finally {
|
|
|
- ozoneContainer.stop();
|
|
|
- }
|
|
|
+ public void forceCloseOpenContainer() throws Exception {
|
|
|
+ // force-close a container that's NOT associated with an open pipeline
|
|
|
+ subject.handle(forceCloseWithoutPipeline(), ozoneContainer, context, null);
|
|
|
+
|
|
|
+ verify(writeChannel, never())
|
|
|
+ .submitRequest(any(), any());
|
|
|
+ verify(containerHandler)
|
|
|
+ .closeContainer(container);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testForceCloseOpenContainer() throws Exception {
|
|
|
- final OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
- final DatanodeDetails datanodeDetails = randomDatanodeDetails();
|
|
|
- final OzoneContainer ozoneContainer =
|
|
|
- getOzoneContainer(conf, datanodeDetails);
|
|
|
- ozoneContainer.start(UUID.randomUUID().toString());
|
|
|
- try {
|
|
|
- final Container container =
|
|
|
- createContainer(conf, datanodeDetails, ozoneContainer);
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(1)).triggerHeartbeat();
|
|
|
- final long containerId = container.getContainerData().getContainerID();
|
|
|
- // A pipeline which doesn't exist in the datanode.
|
|
|
- final PipelineID pipelineId = PipelineID.randomId();
|
|
|
-
|
|
|
- final CloseContainerCommandHandler closeHandler =
|
|
|
- new CloseContainerCommandHandler();
|
|
|
-
|
|
|
- final CloseContainerCommand closeCommand = new CloseContainerCommand(
|
|
|
- containerId, pipelineId, true);
|
|
|
-
|
|
|
- closeHandler.handle(closeCommand, ozoneContainer, context, null);
|
|
|
-
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
|
|
|
- ozoneContainer.getContainerSet().getContainer(containerId)
|
|
|
- .getContainerState());
|
|
|
-
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(3)).triggerHeartbeat();
|
|
|
- } finally {
|
|
|
- ozoneContainer.stop();
|
|
|
- }
|
|
|
+ public void forceCloseOpenContainerWithPipeline() throws Exception {
|
|
|
+ // force-close a container that's associated with an existing pipeline
|
|
|
+ subject.handle(forceCloseWithPipeline(), ozoneContainer, context, null);
|
|
|
+
|
|
|
+ verify(containerHandler)
|
|
|
+ .markContainerForClose(container);
|
|
|
+ verify(writeChannel, never())
|
|
|
+ .submitRequest(any(), any());
|
|
|
+ verify(containerHandler, never())
|
|
|
+ .quasiCloseContainer(container);
|
|
|
+ verify(containerHandler, never())
|
|
|
+ .closeContainer(container);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testQuasiCloseClosedContainer()
|
|
|
- throws Exception {
|
|
|
- final OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
- final DatanodeDetails datanodeDetails = randomDatanodeDetails();
|
|
|
- final OzoneContainer ozoneContainer = getOzoneContainer(
|
|
|
- conf, datanodeDetails);
|
|
|
- ozoneContainer.start(UUID.randomUUID().toString());
|
|
|
- try {
|
|
|
- final Container container = createContainer(
|
|
|
- conf, datanodeDetails, ozoneContainer);
|
|
|
- Mockito.verify(context.getParent(),
|
|
|
- Mockito.times(1)).triggerHeartbeat();
|
|
|
- final long containerId = container.getContainerData().getContainerID();
|
|
|
- final PipelineID pipelineId = PipelineID.valueOf(UUID.fromString(
|
|
|
- container.getContainerData().getOriginPipelineId()));
|
|
|
-
|
|
|
- final CloseContainerCommandHandler closeHandler =
|
|
|
- new CloseContainerCommandHandler();
|
|
|
- final CloseContainerCommand closeCommand = new CloseContainerCommand(
|
|
|
- containerId, pipelineId);
|
|
|
-
|
|
|
- closeHandler.handle(closeCommand, ozoneContainer, context, null);
|
|
|
-
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
|
|
|
- ozoneContainer.getContainerSet().getContainer(containerId)
|
|
|
- .getContainerState());
|
|
|
-
|
|
|
- // The container is closed, now we send close command with
|
|
|
- // pipeline id which doesn't exist.
|
|
|
- // This should cause the datanode to trigger quasi close, since the
|
|
|
- // container is already closed, this should do nothing.
|
|
|
- // The command should not fail either.
|
|
|
- final PipelineID randomPipeline = PipelineID.randomId();
|
|
|
- final CloseContainerCommand quasiCloseCommand =
|
|
|
- new CloseContainerCommand(containerId, randomPipeline);
|
|
|
- closeHandler.handle(quasiCloseCommand, ozoneContainer, context, null);
|
|
|
-
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
|
|
|
- ozoneContainer.getContainerSet().getContainer(containerId)
|
|
|
- .getContainerState());
|
|
|
- } finally {
|
|
|
- ozoneContainer.stop();
|
|
|
- }
|
|
|
+ public void closeAlreadyClosedContainer() throws Exception {
|
|
|
+ container.getContainerData()
|
|
|
+ .setState(ContainerProtos.ContainerDataProto.State.CLOSED);
|
|
|
+
|
|
|
+ // Since the container is already closed, these commands should do nothing,
|
|
|
+ // neither should they fail
|
|
|
+ subject.handle(closeWithUnknownPipeline(), ozoneContainer, context, null);
|
|
|
+ subject.handle(closeWithKnownPipeline(), ozoneContainer, context, null);
|
|
|
+
|
|
|
+ verify(containerHandler, never())
|
|
|
+ .markContainerForClose(container);
|
|
|
+ verify(containerHandler, never())
|
|
|
+ .quasiCloseContainer(container);
|
|
|
+ verify(containerHandler, never())
|
|
|
+ .closeContainer(container);
|
|
|
+ verify(writeChannel, never())
|
|
|
+ .submitRequest(any(), any());
|
|
|
}
|
|
|
|
|
|
- private OzoneContainer getOzoneContainer(final OzoneConfiguration conf,
|
|
|
- final DatanodeDetails datanodeDetails) throws IOException {
|
|
|
- testDir = GenericTestUtils.getTestDir(
|
|
|
- TestCloseContainerCommandHandler.class.getName() + UUID.randomUUID());
|
|
|
- conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getPath());
|
|
|
- conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, testDir.getPath());
|
|
|
- conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true);
|
|
|
- conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, true);
|
|
|
-
|
|
|
- final DatanodeStateMachine datanodeStateMachine = Mockito.mock(
|
|
|
- DatanodeStateMachine.class);
|
|
|
- Mockito.when(datanodeStateMachine.getDatanodeDetails())
|
|
|
- .thenReturn(datanodeDetails);
|
|
|
- Mockito.when(context.getParent()).thenReturn(datanodeStateMachine);
|
|
|
- final OzoneContainer ozoneContainer = new OzoneContainer(
|
|
|
- datanodeDetails, conf, context, null);
|
|
|
- return ozoneContainer;
|
|
|
+ private CloseContainerCommand closeWithKnownPipeline() {
|
|
|
+ return new CloseContainerCommand(CONTAINER_ID, pipelineID);
|
|
|
}
|
|
|
|
|
|
- private Container createContainer(final Configuration conf,
|
|
|
- final DatanodeDetails datanodeDetails,
|
|
|
- final OzoneContainer ozoneContainer) throws Exception {
|
|
|
- final PipelineID pipelineID = PipelineID.randomId();
|
|
|
- final RaftGroupId raftGroupId = RaftGroupId.valueOf(pipelineID.getId());
|
|
|
- final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(conf);
|
|
|
- final RaftPeer peer = RatisHelper.toRaftPeer(datanodeDetails);
|
|
|
- final RaftGroup group = RatisHelper.newRaftGroup(raftGroupId,
|
|
|
- Collections.singleton(datanodeDetails));
|
|
|
- final int maxOutstandingRequests = 100;
|
|
|
- final RaftClient client = RatisHelper
|
|
|
- .newRaftClient(SupportedRpcType.GRPC, peer, retryPolicy,
|
|
|
- maxOutstandingRequests,
|
|
|
- TimeDuration.valueOf(3, TimeUnit.SECONDS));
|
|
|
- Assert.assertTrue(client.groupAdd(group, peer.getId()).isSuccess());
|
|
|
- Thread.sleep(10000);
|
|
|
- final ContainerID containerId = ContainerID.valueof(
|
|
|
- random.nextLong() & Long.MAX_VALUE);
|
|
|
- ContainerProtos.ContainerCommandRequestProto.Builder request =
|
|
|
- ContainerProtos.ContainerCommandRequestProto.newBuilder();
|
|
|
- request.setCmdType(ContainerProtos.Type.CreateContainer);
|
|
|
- request.setContainerID(containerId.getId());
|
|
|
- request.setCreateContainer(
|
|
|
- ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
|
|
|
- request.setDatanodeUuid(datanodeDetails.getUuidString());
|
|
|
- ozoneContainer.getWriteChannel().submitRequest(
|
|
|
- request.build(), pipelineID.getProtobuf());
|
|
|
-
|
|
|
- final Container container = ozoneContainer.getContainerSet().getContainer(
|
|
|
- containerId.getId());
|
|
|
- Assert.assertEquals(ContainerProtos.ContainerDataProto.State.OPEN,
|
|
|
- container.getContainerState());
|
|
|
- return container;
|
|
|
+ private CloseContainerCommand closeWithUnknownPipeline() {
|
|
|
+ return new CloseContainerCommand(CONTAINER_ID, nonExistentPipelineID);
|
|
|
+ }
|
|
|
+
|
|
|
+ private CloseContainerCommand forceCloseWithPipeline() {
|
|
|
+ return new CloseContainerCommand(CONTAINER_ID, pipelineID, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ private CloseContainerCommand forceCloseWithoutPipeline() {
|
|
|
+ return new CloseContainerCommand(CONTAINER_ID, nonExistentPipelineID, true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -339,9 +220,4 @@ public class TestCloseContainerCommandHandler {
|
|
|
.addPort(restPort);
|
|
|
return builder.build();
|
|
|
}
|
|
|
-
|
|
|
- @AfterClass
|
|
|
- public static void teardown() throws IOException {
|
|
|
- FileUtils.deleteDirectory(testDir);
|
|
|
- }
|
|
|
}
|