HDDS-853. Option to force close a container in Datanode.

Contributed by Nanda kumar.
This commit is contained in:
Nanda kumar 2018-11-22 00:49:43 +05:30
parent 892b33e054
commit ebb9245366
4 changed files with 193 additions and 91 deletions

View File

@ -79,8 +79,6 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
final ContainerController controller = ozoneContainer.getController(); final ContainerController controller = ozoneContainer.getController();
final long containerId = closeCommand.getContainerID(); final long containerId = closeCommand.getContainerID();
try { try {
// TODO: Closing of QUASI_CLOSED container.
final Container container = controller.getContainer(containerId); final Container container = controller.getContainer(containerId);
if (container == null) { if (container == null) {
@ -95,6 +93,11 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
// If the container is part of open pipeline, close it via write channel // If the container is part of open pipeline, close it via write channel
if (ozoneContainer.getWriteChannel() if (ozoneContainer.getWriteChannel()
.isExist(closeCommand.getPipelineID())) { .isExist(closeCommand.getPipelineID())) {
if (closeCommand.getForce()) {
LOG.warn("Cannot force close a container when the container is" +
" part of an active pipeline.");
return;
}
ContainerCommandRequestProto request = ContainerCommandRequestProto request =
getContainerCommandRequestProto(datanodeDetails, getContainerCommandRequestProto(datanodeDetails,
closeCommand.getContainerID()); closeCommand.getContainerID());
@ -102,10 +105,14 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
request, closeCommand.getPipelineID()); request, closeCommand.getPipelineID());
return; return;
} }
// If we reach here, there is no active pipeline for this container.
// The container is not part of any open pipeline. if (!closeCommand.getForce()) {
// QUASI_CLOSE the container using ContainerController. // QUASI_CLOSE the container.
controller.quasiCloseContainer(containerId); controller.quasiCloseContainer(containerId);
} else {
// SCM told us to force close the container.
controller.closeContainer(containerId);
}
} catch (NotLeaderException e) { } catch (NotLeaderException e) {
LOG.debug("Follower cannot close container #{}.", containerId); LOG.debug("Follower cannot close container #{}.", containerId);
} catch (IOException e) { } catch (IOException e) {

View File

@ -30,11 +30,18 @@ public class CloseContainerCommand
extends SCMCommand<CloseContainerCommandProto> { extends SCMCommand<CloseContainerCommandProto> {
private final PipelineID pipelineID; private final PipelineID pipelineID;
private boolean force;
public CloseContainerCommand(final long containerID, public CloseContainerCommand(final long containerID,
final PipelineID pipelineID) { final PipelineID pipelineID) {
this(containerID, pipelineID, false);
}
public CloseContainerCommand(final long containerID,
final PipelineID pipelineID, boolean force) {
super(containerID); super(containerID);
this.pipelineID = pipelineID; this.pipelineID = pipelineID;
this.force = force;
} }
/** /**
@ -62,6 +69,7 @@ public CloseContainerCommandProto getProto() {
.setContainerID(getId()) .setContainerID(getId())
.setCmdId(getId()) .setCmdId(getId())
.setPipelineID(pipelineID.getProtobuf()) .setPipelineID(pipelineID.getProtobuf())
.setForce(force)
.build(); .build();
} }
@ -69,7 +77,8 @@ public static CloseContainerCommand getFromProtobuf(
CloseContainerCommandProto closeContainerProto) { CloseContainerCommandProto closeContainerProto) {
Preconditions.checkNotNull(closeContainerProto); Preconditions.checkNotNull(closeContainerProto);
return new CloseContainerCommand(closeContainerProto.getCmdId(), return new CloseContainerCommand(closeContainerProto.getCmdId(),
PipelineID.getFromProtobuf(closeContainerProto.getPipelineID())); PipelineID.getFromProtobuf(closeContainerProto.getPipelineID()),
closeContainerProto.getForce());
} }
public long getContainerID() { public long getContainerID() {

View File

@ -293,6 +293,8 @@ message CloseContainerCommandProto {
required PipelineID pipelineID = 2; required PipelineID pipelineID = 2;
// cmdId will be removed // cmdId will be removed
required int64 cmdId = 3; required int64 cmdId = 3;
// Force will be used when closing a container out side of ratis.
optional bool force = 4 [default = false];
} }
/** /**

View File

@ -17,6 +17,7 @@
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
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;
@ -24,7 +25,9 @@
import org.apache.hadoop.hdds.scm.ScmConfigKeys; 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.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine; import org.apache.hadoop.ozone.container.common.interfaces.Container;
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.statemachine.StateContext;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
@ -44,6 +47,7 @@
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Random;
import java.util.UUID; import java.util.UUID;
/** /**
@ -52,8 +56,156 @@
public class TestCloseContainerCommandHandler { public class TestCloseContainerCommandHandler {
private final StateContext context = Mockito.mock(StateContext.class); private final StateContext context = Mockito.mock(StateContext.class);
private final Random random = new Random();
private static File testDir; private static File testDir;
@Test
public void testCloseContainerViaRatis()
throws Exception {
final OzoneConfiguration conf = new OzoneConfiguration();
final DatanodeDetails datanodeDetails = randomDatanodeDetails();
final OzoneContainer ozoneContainer =
getOzoneContainer(conf, datanodeDetails);
ozoneContainer.start();
try {
final Container container =
createContainer(conf, datanodeDetails, ozoneContainer);
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(2)).triggerHeartbeat();
} finally {
ozoneContainer.stop();
}
}
@Test
public void testCloseContainerViaStandalone()
throws Exception {
final OzoneConfiguration conf = new OzoneConfiguration();
final DatanodeDetails datanodeDetails = randomDatanodeDetails();
final OzoneContainer ozoneContainer =
getOzoneContainer(conf, datanodeDetails);
ozoneContainer.start();
try {
final Container container =
createContainer(conf, datanodeDetails, ozoneContainer);
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(2)).triggerHeartbeat();
} finally {
ozoneContainer.stop();
}
}
@Test
public void testQuasiCloseToClose() throws Exception {
final OzoneConfiguration conf = new OzoneConfiguration();
final DatanodeDetails datanodeDetails = randomDatanodeDetails();
final OzoneContainer ozoneContainer =
getOzoneContainer(conf, datanodeDetails);
ozoneContainer.start();
try {
final Container container =
createContainer(conf, datanodeDetails, ozoneContainer);
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(2)).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(3)).triggerHeartbeat();
} finally {
ozoneContainer.stop();
}
}
@Test
public void testForceCloseOpenContainer() throws Exception {
final OzoneConfiguration conf = new OzoneConfiguration();
final DatanodeDetails datanodeDetails = randomDatanodeDetails();
final OzoneContainer ozoneContainer =
getOzoneContainer(conf, datanodeDetails);
ozoneContainer.start();
try {
final Container container =
createContainer(conf, datanodeDetails, ozoneContainer);
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(2)).triggerHeartbeat();
} finally {
ozoneContainer.stop();
}
}
private OzoneContainer getOzoneContainer(final OzoneConfiguration conf, private OzoneContainer getOzoneContainer(final OzoneConfiguration conf,
final DatanodeDetails datanodeDetails) throws IOException { final DatanodeDetails datanodeDetails) throws IOException {
@ -67,19 +219,15 @@ private OzoneContainer getOzoneContainer(final OzoneConfiguration conf,
Mockito.when(datanodeStateMachine.getDatanodeDetails()) Mockito.when(datanodeStateMachine.getDatanodeDetails())
.thenReturn(datanodeDetails); .thenReturn(datanodeDetails);
Mockito.when(context.getParent()).thenReturn(datanodeStateMachine); Mockito.when(context.getParent()).thenReturn(datanodeStateMachine);
return new OzoneContainer(datanodeDetails, conf, context); final OzoneContainer ozoneContainer = new OzoneContainer(
datanodeDetails, conf, context);
ozoneContainer.getDispatcher().setScmId(UUID.randomUUID().toString());
return ozoneContainer;
} }
private Container createContainer(final Configuration conf,
@Test final DatanodeDetails datanodeDetails,
public void testCloseContainerViaRatis() final OzoneContainer ozoneContainer) throws Exception {
throws IOException, InterruptedException {
final OzoneConfiguration conf = new OzoneConfiguration();
final DatanodeDetails datanodeDetails = randomDatanodeDetails();
final OzoneContainer container = getOzoneContainer(conf, datanodeDetails);
container.getDispatcher().setScmId(UUID.randomUUID().toString());
container.start();
// Give some time for ratis for leader election.
final PipelineID pipelineID = PipelineID.randomId(); final PipelineID pipelineID = PipelineID.randomId();
final RaftGroupId raftGroupId = RaftGroupId.valueOf(pipelineID.getId()); final RaftGroupId raftGroupId = RaftGroupId.valueOf(pipelineID.getId());
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(conf); final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(conf);
@ -88,9 +236,10 @@ public void testCloseContainerViaRatis()
Collections.singleton(datanodeDetails)); Collections.singleton(datanodeDetails));
final RaftClient client = RatisHelper.newRaftClient( final RaftClient client = RatisHelper.newRaftClient(
SupportedRpcType.GRPC, peer, retryPolicy); SupportedRpcType.GRPC, peer, retryPolicy);
System.out.println(client.groupAdd(group, peer.getId()).isSuccess()); Assert.assertTrue(client.groupAdd(group, peer.getId()).isSuccess());
Thread.sleep(2000); Thread.sleep(2000);
final ContainerID containerId = ContainerID.valueof(1); final ContainerID containerId = ContainerID.valueof(
random.nextLong() & Long.MAX_VALUE);
ContainerProtos.ContainerCommandRequestProto.Builder request = ContainerProtos.ContainerCommandRequestProto.Builder request =
ContainerProtos.ContainerCommandRequestProto.newBuilder(); ContainerProtos.ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.CreateContainer); request.setCmdType(ContainerProtos.Type.CreateContainer);
@ -99,79 +248,14 @@ public void testCloseContainerViaRatis()
ContainerProtos.CreateContainerRequestProto.getDefaultInstance()); ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
request.setTraceID(UUID.randomUUID().toString()); request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(datanodeDetails.getUuidString()); request.setDatanodeUuid(datanodeDetails.getUuidString());
container.getWriteChannel().submitRequest( ozoneContainer.getWriteChannel().submitRequest(
request.build(), pipelineID.getProtobuf()); request.build(), pipelineID.getProtobuf());
final Container container = ozoneContainer.getContainerSet().getContainer(
containerId.getId());
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.OPEN, Assert.assertEquals(ContainerProtos.ContainerDataProto.State.OPEN,
container.getContainerSet().getContainer( container.getContainerState());
containerId.getId()).getContainerState()); return container;
// We have created a container via ratis. Now close the container on ratis.
final CloseContainerCommandHandler closeHandler =
new CloseContainerCommandHandler();
final CloseContainerCommand command = new CloseContainerCommand(
containerId.getId(), pipelineID);
closeHandler.handle(command, container, context, null);
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
container.getContainerSet().getContainer(
containerId.getId()).getContainerState());
Mockito.verify(context.getParent(), Mockito.times(2)).triggerHeartbeat();
container.stop();
}
@Test
public void testCloseContainerViaStandalone()
throws IOException, InterruptedException {
final OzoneConfiguration conf = new OzoneConfiguration();
final DatanodeDetails datanodeDetails = randomDatanodeDetails();
final OzoneContainer container = getOzoneContainer(conf, datanodeDetails);
container.getDispatcher().setScmId(UUID.randomUUID().toString());
container.start();
// Give some time for ratis for leader election.
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 RaftClient client = RatisHelper.newRaftClient(
SupportedRpcType.GRPC, peer, retryPolicy);
System.out.println(client.groupAdd(group, peer.getId()).isSuccess());
Thread.sleep(2000);
final ContainerID containerId = ContainerID.valueof(2);
ContainerProtos.ContainerCommandRequestProto.Builder request =
ContainerProtos.ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.CreateContainer);
request.setContainerID(containerId.getId());
request.setCreateContainer(
ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(datanodeDetails.getUuidString());
container.getWriteChannel().submitRequest(
request.build(), pipelineID.getProtobuf());
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.OPEN,
container.getContainerSet().getContainer(
containerId.getId()).getContainerState());
// We have created a container via ratis. Now quasi close it
final CloseContainerCommandHandler closeHandler =
new CloseContainerCommandHandler();
// Specify a pipeline which doesn't exist in the datanode.
final CloseContainerCommand command = new CloseContainerCommand(
containerId.getId(), PipelineID.randomId());
closeHandler.handle(command, container, context, null);
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.QUASI_CLOSED,
container.getContainerSet().getContainer(
containerId.getId()).getContainerState());
Mockito.verify(context.getParent(), Mockito.times(2)).triggerHeartbeat();
container.stop();
} }
/** /**