HDDS-1413. Attempt to fix TestCloseContainerCommandHandler by adjusting timeouts

Signed-off-by: Anu Engineer <aengineer@apache.org>
This commit is contained in:
Doroszlai, Attila 2019-08-29 18:01:21 +02:00 committed by Anu Engineer
parent c4411f7fdf
commit a2d083f2c5
1 changed files with 133 additions and 257 deletions

View File

@ -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;
@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()));
private OzoneContainer ozoneContainer;
private StateContext context;
private XceiverServerSpi writeChannel;
private Container container;
private Handler containerHandler;
private PipelineID pipelineID;
private PipelineID nonExistentPipelineID = PipelineID.randomId();
// 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);
private CloseContainerCommandHandler subject =
new CloseContainerCommandHandler();
closeHandler.handle(command, ozoneContainer, context, null);
@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);
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
ozoneContainer.getContainerSet().getContainer(containerId)
.getContainerState());
pipelineID = PipelineID.randomId();
Mockito.verify(context.getParent(),
Mockito.times(3)).triggerHeartbeat();
} finally {
ozoneContainer.stop();
}
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 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();
public void closeContainerWithPipeline() throws Exception {
// close a container that's associated with an existing pipeline
subject.handle(closeWithKnownPipeline(), ozoneContainer, context, null);
// 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();
}
verify(containerHandler)
.markContainerForClose(container);
verify(writeChannel)
.submitRequest(any(), eq(pipelineID.getProtobuf()));
verify(containerHandler, never())
.quasiCloseContainer(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();
public void closeContainerWithoutPipeline() throws IOException {
// close a container that's NOT associated with an open pipeline
subject.handle(closeWithUnknownPipeline(), ozoneContainer, context, null);
// 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();
}
verify(containerHandler)
.markContainerForClose(container);
verify(writeChannel, never())
.submitRequest(any(), any());
verify(containerHandler)
.quasiCloseContainer(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();
public void forceCloseQuasiClosedContainer() throws Exception {
// force-close a container that's already quasi closed
container.getContainerData()
.setState(ContainerProtos.ContainerDataProto.State.QUASI_CLOSED);
final CloseContainerCommandHandler closeHandler =
new CloseContainerCommandHandler();
subject.handle(forceCloseWithoutPipeline(), ozoneContainer, context, null);
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();
}
verify(writeChannel, never())
.submitRequest(any(), any());
verify(containerHandler)
.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()));
public void forceCloseOpenContainer() throws Exception {
// force-close a container that's NOT associated with an open pipeline
subject.handle(forceCloseWithoutPipeline(), ozoneContainer, context, null);
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();
}
verify(writeChannel, never())
.submitRequest(any(), any());
verify(containerHandler)
.closeContainer(container);
}
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);
@Test
public void forceCloseOpenContainerWithPipeline() throws Exception {
// force-close a container that's associated with an existing pipeline
subject.handle(forceCloseWithPipeline(), ozoneContainer, context, null);
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;
verify(containerHandler)
.markContainerForClose(container);
verify(writeChannel, never())
.submitRequest(any(), any());
verify(containerHandler, never())
.quasiCloseContainer(container);
verify(containerHandler, never())
.closeContainer(container);
}
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());
@Test
public void closeAlreadyClosedContainer() throws Exception {
container.getContainerData()
.setState(ContainerProtos.ContainerDataProto.State.CLOSED);
final Container container = ozoneContainer.getContainerSet().getContainer(
containerId.getId());
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.OPEN,
container.getContainerState());
return container;
// 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 CloseContainerCommand closeWithKnownPipeline() {
return new CloseContainerCommand(CONTAINER_ID, pipelineID);
}
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 @@ private static DatanodeDetails randomDatanodeDetails() {
.addPort(restPort);
return builder.build();
}
@AfterClass
public static void teardown() throws IOException {
FileUtils.deleteDirectory(testDir);
}
}