HDDS-1561: Mark OPEN containers as QUASI_CLOSED as part of Ratis groupRemove (#1401)
This commit is contained in:
parent
494d75eb2b
commit
6e4cdf89ef
|
@ -284,6 +284,14 @@ public abstract class ContainerData {
|
||||||
return ContainerDataProto.State.QUASI_CLOSED == state;
|
return ContainerDataProto.State.QUASI_CLOSED == state;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* checks if the container is unhealthy.
|
||||||
|
* @return - boolean
|
||||||
|
*/
|
||||||
|
public synchronized boolean isUnhealthy() {
|
||||||
|
return ContainerDataProto.State.UNHEALTHY == state;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Marks this container as quasi closed.
|
* Marks this container as quasi closed.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -86,37 +86,38 @@ public class CloseContainerCommandHandler implements CommandHandler {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (container.getContainerState() ==
|
// move the container to CLOSING if in OPEN state
|
||||||
ContainerProtos.ContainerDataProto.State.CLOSED) {
|
|
||||||
// Closing a container is an idempotent operation.
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Move the container to CLOSING state
|
|
||||||
controller.markContainerForClose(containerId);
|
controller.markContainerForClose(containerId);
|
||||||
|
|
||||||
// If the container is part of open pipeline, close it via write channel
|
switch (container.getContainerState()) {
|
||||||
if (ozoneContainer.getWriteChannel()
|
case OPEN:
|
||||||
.isExist(closeCommand.getPipelineID())) {
|
case CLOSING:
|
||||||
if (closeCommand.getForce()) {
|
// If the container is part of open pipeline, close it via write channel
|
||||||
LOG.warn("Cannot force close a container when the container is" +
|
if (ozoneContainer.getWriteChannel()
|
||||||
" part of an active pipeline.");
|
.isExist(closeCommand.getPipelineID())) {
|
||||||
return;
|
ContainerCommandRequestProto request =
|
||||||
|
getContainerCommandRequestProto(datanodeDetails,
|
||||||
|
closeCommand.getContainerID());
|
||||||
|
ozoneContainer.getWriteChannel()
|
||||||
|
.submitRequest(request, closeCommand.getPipelineID());
|
||||||
|
} else {
|
||||||
|
// Container should not exist in CLOSING state without a pipeline
|
||||||
|
controller.markContainerUnhealthy(containerId);
|
||||||
}
|
}
|
||||||
ContainerCommandRequestProto request =
|
break;
|
||||||
getContainerCommandRequestProto(datanodeDetails,
|
case QUASI_CLOSED:
|
||||||
closeCommand.getContainerID());
|
if (closeCommand.getForce()) {
|
||||||
ozoneContainer.getWriteChannel().submitRequest(
|
controller.closeContainer(containerId);
|
||||||
request, closeCommand.getPipelineID());
|
break;
|
||||||
return;
|
}
|
||||||
}
|
case CLOSED:
|
||||||
// If we reach here, there is no active pipeline for this container.
|
break;
|
||||||
if (!closeCommand.getForce()) {
|
case UNHEALTHY:
|
||||||
// QUASI_CLOSE the container.
|
case INVALID:
|
||||||
controller.quasiCloseContainer(containerId);
|
LOG.debug("Cannot close the container #{}, the container is"
|
||||||
} else {
|
+ " in {} state.", containerId, container.getContainerState());
|
||||||
// SCM told us to force close the container.
|
default:
|
||||||
controller.closeContainer(containerId);
|
break;
|
||||||
}
|
}
|
||||||
} catch (NotLeaderException e) {
|
} catch (NotLeaderException e) {
|
||||||
LOG.debug("Follower cannot close container #{}.", containerId);
|
LOG.debug("Follower cannot close container #{}.", containerId);
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenExcep
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||||
|
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
|
||||||
import org.apache.hadoop.util.Time;
|
import org.apache.hadoop.util.Time;
|
||||||
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
|
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
|
||||||
import org.apache.ratis.protocol.RaftGroupId;
|
import org.apache.ratis.protocol.RaftGroupId;
|
||||||
|
@ -138,6 +139,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
new SimpleStateMachineStorage();
|
new SimpleStateMachineStorage();
|
||||||
private final RaftGroupId gid;
|
private final RaftGroupId gid;
|
||||||
private final ContainerDispatcher dispatcher;
|
private final ContainerDispatcher dispatcher;
|
||||||
|
private final ContainerController containerController;
|
||||||
private ThreadPoolExecutor chunkExecutor;
|
private ThreadPoolExecutor chunkExecutor;
|
||||||
private final XceiverServerRatis ratisServer;
|
private final XceiverServerRatis ratisServer;
|
||||||
private final ConcurrentHashMap<Long,
|
private final ConcurrentHashMap<Long,
|
||||||
|
@ -160,11 +162,13 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
|
|
||||||
@SuppressWarnings("parameternumber")
|
@SuppressWarnings("parameternumber")
|
||||||
public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher,
|
public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher,
|
||||||
ThreadPoolExecutor chunkExecutor, XceiverServerRatis ratisServer,
|
ContainerController containerController, ThreadPoolExecutor chunkExecutor,
|
||||||
long expiryInterval, boolean isBlockTokenEnabled,
|
XceiverServerRatis ratisServer, long expiryInterval,
|
||||||
TokenVerifier tokenVerifier, Configuration conf) {
|
boolean isBlockTokenEnabled, TokenVerifier tokenVerifier,
|
||||||
|
Configuration conf) {
|
||||||
this.gid = gid;
|
this.gid = gid;
|
||||||
this.dispatcher = dispatcher;
|
this.dispatcher = dispatcher;
|
||||||
|
this.containerController = containerController;
|
||||||
this.chunkExecutor = chunkExecutor;
|
this.chunkExecutor = chunkExecutor;
|
||||||
this.ratisServer = ratisServer;
|
this.ratisServer = ratisServer;
|
||||||
metrics = CSMMetrics.create(gid);
|
metrics = CSMMetrics.create(gid);
|
||||||
|
@ -215,6 +219,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super.initialize(server, id, raftStorage);
|
super.initialize(server, id, raftStorage);
|
||||||
storage.init(raftStorage);
|
storage.init(raftStorage);
|
||||||
|
ratisServer.notifyGroupAdd(gid);
|
||||||
|
|
||||||
loadSnapshot(storage.getLatestSnapshot());
|
loadSnapshot(storage.getLatestSnapshot());
|
||||||
}
|
}
|
||||||
|
@ -800,6 +805,21 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
return future;
|
return future;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void notifyGroupRemove() {
|
||||||
|
ratisServer.notifyGroupRemove(gid);
|
||||||
|
// Make best effort to quasi-close all the containers on group removal.
|
||||||
|
// Containers already in terminal state like CLOSED or UNHEALTHY will not
|
||||||
|
// be affected.
|
||||||
|
for (Long cid : createContainerSet) {
|
||||||
|
try {
|
||||||
|
containerController.markContainerForClose(cid);
|
||||||
|
containerController.quasiCloseContainer(cid);
|
||||||
|
} catch (IOException e) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
evictStateMachineCache();
|
evictStateMachineCache();
|
||||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
|
||||||
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
|
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
|
||||||
|
|
||||||
import io.opentracing.Scope;
|
import io.opentracing.Scope;
|
||||||
|
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
|
||||||
import org.apache.ratis.RaftConfigKeys;
|
import org.apache.ratis.RaftConfigKeys;
|
||||||
import org.apache.hadoop.hdds.ratis.RatisHelper;
|
import org.apache.hadoop.hdds.ratis.RatisHelper;
|
||||||
import org.apache.ratis.conf.RaftProperties;
|
import org.apache.ratis.conf.RaftProperties;
|
||||||
|
@ -63,9 +64,11 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.concurrent.ArrayBlockingQueue;
|
import java.util.concurrent.ArrayBlockingQueue;
|
||||||
|
@ -90,6 +93,7 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
private final RaftServer server;
|
private final RaftServer server;
|
||||||
private ThreadPoolExecutor chunkExecutor;
|
private ThreadPoolExecutor chunkExecutor;
|
||||||
private final ContainerDispatcher dispatcher;
|
private final ContainerDispatcher dispatcher;
|
||||||
|
private final ContainerController containerController;
|
||||||
private ClientId clientId = ClientId.randomId();
|
private ClientId clientId = ClientId.randomId();
|
||||||
private final StateContext context;
|
private final StateContext context;
|
||||||
private final ReplicationLevel replicationLevel;
|
private final ReplicationLevel replicationLevel;
|
||||||
|
@ -98,10 +102,15 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
private boolean isStarted = false;
|
private boolean isStarted = false;
|
||||||
private DatanodeDetails datanodeDetails;
|
private DatanodeDetails datanodeDetails;
|
||||||
private final Configuration conf;
|
private final Configuration conf;
|
||||||
|
// TODO: Remove the gids set when Ratis supports an api to query active
|
||||||
|
// pipelines
|
||||||
|
private final Set<RaftGroupId> raftGids = new HashSet<>();
|
||||||
|
|
||||||
|
@SuppressWarnings("parameternumber")
|
||||||
private XceiverServerRatis(DatanodeDetails dd, int port,
|
private XceiverServerRatis(DatanodeDetails dd, int port,
|
||||||
ContainerDispatcher dispatcher, Configuration conf, StateContext
|
ContainerDispatcher dispatcher, ContainerController containerController,
|
||||||
context, GrpcTlsConfig tlsConfig, CertificateClient caClient)
|
StateContext context, GrpcTlsConfig tlsConfig, CertificateClient caClient,
|
||||||
|
Configuration conf)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(conf, caClient);
|
super(conf, caClient);
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
|
@ -127,6 +136,7 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL_DEFAULT,
|
DFS_CONTAINER_RATIS_STATEMACHINEDATA_CACHE_EXPIRY_INTERVAL_DEFAULT,
|
||||||
TimeUnit.MILLISECONDS);
|
TimeUnit.MILLISECONDS);
|
||||||
this.dispatcher = dispatcher;
|
this.dispatcher = dispatcher;
|
||||||
|
this.containerController = containerController;
|
||||||
|
|
||||||
RaftServer.Builder builder =
|
RaftServer.Builder builder =
|
||||||
RaftServer.newBuilder().setServerId(RatisHelper.toRaftPeerId(dd))
|
RaftServer.newBuilder().setServerId(RatisHelper.toRaftPeerId(dd))
|
||||||
|
@ -139,9 +149,10 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerStateMachine getStateMachine(RaftGroupId gid) {
|
private ContainerStateMachine getStateMachine(RaftGroupId gid) {
|
||||||
return new ContainerStateMachine(gid, dispatcher, chunkExecutor, this,
|
return new ContainerStateMachine(gid, dispatcher, containerController,
|
||||||
cacheEntryExpiryInteval, getSecurityConfig().isBlockTokenEnabled(),
|
chunkExecutor, this, cacheEntryExpiryInteval,
|
||||||
getBlockTokenVerifier(), conf);
|
getSecurityConfig().isBlockTokenEnabled(), getBlockTokenVerifier(),
|
||||||
|
conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
private RaftProperties newRaftProperties() {
|
private RaftProperties newRaftProperties() {
|
||||||
|
@ -258,7 +269,7 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
.getDuration(), timeUnit);
|
.getDuration(), timeUnit);
|
||||||
final TimeDuration nodeFailureTimeout =
|
final TimeDuration nodeFailureTimeout =
|
||||||
TimeDuration.valueOf(duration, timeUnit);
|
TimeDuration.valueOf(duration, timeUnit);
|
||||||
RaftServerConfigKeys.setLeaderElectionTimeout(properties,
|
RaftServerConfigKeys.Notification.setNoLeaderTimeout(properties,
|
||||||
nodeFailureTimeout);
|
nodeFailureTimeout);
|
||||||
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
|
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
|
||||||
nodeFailureTimeout);
|
nodeFailureTimeout);
|
||||||
|
@ -367,8 +378,8 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
|
|
||||||
public static XceiverServerRatis newXceiverServerRatis(
|
public static XceiverServerRatis newXceiverServerRatis(
|
||||||
DatanodeDetails datanodeDetails, Configuration ozoneConf,
|
DatanodeDetails datanodeDetails, Configuration ozoneConf,
|
||||||
ContainerDispatcher dispatcher, StateContext context,
|
ContainerDispatcher dispatcher, ContainerController containerController,
|
||||||
CertificateClient caClient) throws IOException {
|
CertificateClient caClient, StateContext context) throws IOException {
|
||||||
int localPort = ozoneConf.getInt(
|
int localPort = ozoneConf.getInt(
|
||||||
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
|
||||||
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT_DEFAULT);
|
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT_DEFAULT);
|
||||||
|
@ -383,8 +394,8 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
GrpcTlsConfig tlsConfig = RatisHelper.createTlsServerConfig(
|
GrpcTlsConfig tlsConfig = RatisHelper.createTlsServerConfig(
|
||||||
new SecurityConfig(ozoneConf));
|
new SecurityConfig(ozoneConf));
|
||||||
|
|
||||||
return new XceiverServerRatis(datanodeDetails, localPort,
|
return new XceiverServerRatis(datanodeDetails, localPort, dispatcher,
|
||||||
dispatcher, ozoneConf, context, tlsConfig, caClient);
|
containerController, context, tlsConfig, caClient, ozoneConf);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -561,13 +572,8 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isExist(HddsProtos.PipelineID pipelineId) {
|
public boolean isExist(HddsProtos.PipelineID pipelineId) {
|
||||||
for (RaftGroupId groupId : server.getGroupIds()) {
|
return raftGids.contains(
|
||||||
if (PipelineID.valueOf(groupId.getUuid()).getProtobuf()
|
RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineId).getId()));
|
||||||
.equals(pipelineId)) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -658,4 +664,12 @@ public final class XceiverServerRatis extends XceiverServer {
|
||||||
minIndex = RatisHelper.getMinReplicatedIndex(reply.getCommitInfos());
|
minIndex = RatisHelper.getMinReplicatedIndex(reply.getCommitInfos());
|
||||||
return minIndex == null ? -1 : minIndex.longValue();
|
return minIndex == null ? -1 : minIndex.longValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void notifyGroupRemove(RaftGroupId gid) {
|
||||||
|
raftGids.remove(gid);
|
||||||
|
}
|
||||||
|
|
||||||
|
void notifyGroupAdd(RaftGroupId gid) {
|
||||||
|
raftGids.add(gid);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -113,7 +113,8 @@ public class OzoneContainer {
|
||||||
*/
|
*/
|
||||||
this.controller = new ContainerController(containerSet, handlers);
|
this.controller = new ContainerController(containerSet, handlers);
|
||||||
this.writeChannel = XceiverServerRatis.newXceiverServerRatis(
|
this.writeChannel = XceiverServerRatis.newXceiverServerRatis(
|
||||||
datanodeDetails, config, hddsDispatcher, context, certClient);
|
datanodeDetails, config, hddsDispatcher, controller, certClient,
|
||||||
|
context);
|
||||||
this.readChannel = new XceiverServerGrpc(
|
this.readChannel = new XceiverServerGrpc(
|
||||||
datanodeDetails, config, hddsDispatcher, certClient,
|
datanodeDetails, config, hddsDispatcher, certClient,
|
||||||
createReplicationService());
|
createReplicationService());
|
||||||
|
|
|
@ -119,8 +119,10 @@ public class TestCloseContainerCommandHandler {
|
||||||
.markContainerForClose(container);
|
.markContainerForClose(container);
|
||||||
verify(writeChannel, never())
|
verify(writeChannel, never())
|
||||||
.submitRequest(any(), any());
|
.submitRequest(any(), any());
|
||||||
|
// Container in CLOSING state is moved to UNHEALTHY if pipeline does not
|
||||||
|
// exist. Container should not exist in CLOSING state without a pipeline.
|
||||||
verify(containerHandler)
|
verify(containerHandler)
|
||||||
.quasiCloseContainer(container);
|
.markContainerUnhealthy(container);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -144,8 +146,10 @@ public class TestCloseContainerCommandHandler {
|
||||||
|
|
||||||
verify(writeChannel, never())
|
verify(writeChannel, never())
|
||||||
.submitRequest(any(), any());
|
.submitRequest(any(), any());
|
||||||
|
// Container in CLOSING state is moved to UNHEALTHY if pipeline does not
|
||||||
|
// exist. Container should not exist in CLOSING state without a pipeline.
|
||||||
verify(containerHandler)
|
verify(containerHandler)
|
||||||
.closeContainer(container);
|
.markContainerUnhealthy(container);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -155,7 +159,7 @@ public class TestCloseContainerCommandHandler {
|
||||||
|
|
||||||
verify(containerHandler)
|
verify(containerHandler)
|
||||||
.markContainerForClose(container);
|
.markContainerForClose(container);
|
||||||
verify(writeChannel, never())
|
verify(writeChannel)
|
||||||
.submitRequest(any(), any());
|
.submitRequest(any(), any());
|
||||||
verify(containerHandler, never())
|
verify(containerHandler, never())
|
||||||
.quasiCloseContainer(container);
|
.quasiCloseContainer(container);
|
||||||
|
|
|
@ -48,7 +48,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
<hdds.version>0.5.0-SNAPSHOT</hdds.version>
|
<hdds.version>0.5.0-SNAPSHOT</hdds.version>
|
||||||
|
|
||||||
<!-- Apache Ratis version -->
|
<!-- Apache Ratis version -->
|
||||||
<ratis.version>0.4.0-2337318-SNAPSHOT</ratis.version>
|
<ratis.version>0.4.0-78e95b9-SNAPSHOT</ratis.version>
|
||||||
|
|
||||||
<bouncycastle.version>1.60</bouncycastle.version>
|
<bouncycastle.version>1.60</bouncycastle.version>
|
||||||
|
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
/**
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
* contributor license agreements. See the NOTICE file distributed with this
|
* contributor license agreements. See the NOTICE file distributed with this
|
||||||
* work for additional information regarding copyright ownership. The ASF
|
* work for additional information regarding copyright ownership. The ASF
|
||||||
|
@ -26,7 +26,6 @@ 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.container.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
|
||||||
import org.apache.hadoop.ozone.HddsDatanodeService;
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.client.ObjectStore;
|
import org.apache.hadoop.ozone.client.ObjectStore;
|
||||||
|
@ -97,10 +96,11 @@ public class TestCloseContainerByPipeline {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
|
public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
|
||||||
|
String keyName = "testIfCloseContainerCommandHandlerIsInvoked";
|
||||||
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
||||||
.createKey("standalone", 1024, ReplicationType.RATIS,
|
.createKey(keyName, 1024, ReplicationType.RATIS, ReplicationFactor.ONE,
|
||||||
ReplicationFactor.ONE, new HashMap<>());
|
new HashMap<>());
|
||||||
key.write("standalone".getBytes());
|
key.write(keyName.getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
//get the name of a valid container
|
//get the name of a valid container
|
||||||
|
@ -108,7 +108,7 @@ public class TestCloseContainerByPipeline {
|
||||||
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
||||||
.setType(HddsProtos.ReplicationType.RATIS)
|
.setType(HddsProtos.ReplicationType.RATIS)
|
||||||
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
||||||
.setKeyName("standalone").setRefreshPipeline(true).build();
|
.setKeyName(keyName).setRefreshPipeline(true).build();
|
||||||
OmKeyLocationInfo omKeyLocationInfo =
|
OmKeyLocationInfo omKeyLocationInfo =
|
||||||
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
||||||
.get(0).getBlocksLatestVersionOnly().get(0);
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
||||||
|
@ -186,16 +186,21 @@ public class TestCloseContainerByPipeline {
|
||||||
// the container will not be closed via RATIS
|
// the container will not be closed via RATIS
|
||||||
cluster.getStorageContainerManager().getScmNodeManager()
|
cluster.getStorageContainerManager().getScmNodeManager()
|
||||||
.addDatanodeCommand(datanodeDetails.getUuid(),
|
.addDatanodeCommand(datanodeDetails.getUuid(),
|
||||||
new CloseContainerCommand(containerID, PipelineID.randomId()));
|
new CloseContainerCommand(containerID, pipeline.getId()));
|
||||||
|
|
||||||
//double check if it's really closed (waitFor also throws an exception)
|
//double check if it's really closed (waitFor also throws an exception)
|
||||||
// TODO: change the below line after implementing QUASI_CLOSED to CLOSED
|
// TODO: change the below line after implementing QUASI_CLOSED to CLOSED
|
||||||
// logic. The container will be QUASI closed as of now
|
// logic. The container will be QUASI closed as of now
|
||||||
GenericTestUtils
|
GenericTestUtils
|
||||||
.waitFor(() -> isContainerQuasiClosed(
|
.waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails),
|
||||||
cluster, containerID, datanodeDetails), 500, 5 * 1000);
|
500, 5 * 1000);
|
||||||
Assert.assertTrue(
|
Assert.assertTrue(isContainerClosed(cluster, containerID, datanodeDetails));
|
||||||
isContainerQuasiClosed(cluster, containerID, datanodeDetails));
|
|
||||||
|
cluster.getStorageContainerManager().getPipelineManager()
|
||||||
|
.finalizeAndDestroyPipeline(pipeline, false);
|
||||||
|
Thread.sleep(5000);
|
||||||
|
// Pipeline close should not affect a container in CLOSED state
|
||||||
|
Assert.assertTrue(isContainerClosed(cluster, containerID, datanodeDetails));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -258,6 +263,65 @@ public class TestCloseContainerByPipeline {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testQuasiCloseTransitionViaRatis()
|
||||||
|
throws IOException, TimeoutException, InterruptedException {
|
||||||
|
|
||||||
|
String keyName = "testQuasiCloseTransitionViaRatis";
|
||||||
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
||||||
|
.createKey(keyName, 1024, ReplicationType.RATIS,
|
||||||
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
|
key.write(keyName.getBytes());
|
||||||
|
key.close();
|
||||||
|
|
||||||
|
OmKeyArgs keyArgs =
|
||||||
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
||||||
|
.setType(HddsProtos.ReplicationType.RATIS)
|
||||||
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
||||||
|
.setKeyName(keyName)
|
||||||
|
.setRefreshPipeline(true)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
OmKeyLocationInfo omKeyLocationInfo =
|
||||||
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
||||||
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
||||||
|
|
||||||
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
|
ContainerInfo container = cluster.getStorageContainerManager()
|
||||||
|
.getContainerManager().getContainer(ContainerID.valueof(containerID));
|
||||||
|
Pipeline pipeline = cluster.getStorageContainerManager()
|
||||||
|
.getPipelineManager().getPipeline(container.getPipelineID());
|
||||||
|
List<DatanodeDetails> datanodes = pipeline.getNodes();
|
||||||
|
Assert.assertEquals(datanodes.size(), 1);
|
||||||
|
|
||||||
|
DatanodeDetails datanodeDetails = datanodes.get(0);
|
||||||
|
Assert
|
||||||
|
.assertFalse(isContainerClosed(cluster, containerID, datanodeDetails));
|
||||||
|
|
||||||
|
// close the pipeline
|
||||||
|
cluster.getStorageContainerManager()
|
||||||
|
.getPipelineManager().finalizeAndDestroyPipeline(pipeline, false);
|
||||||
|
|
||||||
|
// All the containers in OPEN or CLOSING state should transition to
|
||||||
|
// QUASI-CLOSED after pipeline close
|
||||||
|
GenericTestUtils.waitFor(
|
||||||
|
() -> isContainerQuasiClosed(cluster, containerID, datanodeDetails),
|
||||||
|
500, 5 * 1000);
|
||||||
|
Assert.assertTrue(
|
||||||
|
isContainerQuasiClosed(cluster, containerID, datanodeDetails));
|
||||||
|
|
||||||
|
// Send close container command from SCM to datanode with forced flag as
|
||||||
|
// true
|
||||||
|
cluster.getStorageContainerManager().getScmNodeManager()
|
||||||
|
.addDatanodeCommand(datanodeDetails.getUuid(),
|
||||||
|
new CloseContainerCommand(containerID, pipeline.getId(), true));
|
||||||
|
GenericTestUtils
|
||||||
|
.waitFor(() -> isContainerClosed(
|
||||||
|
cluster, containerID, datanodeDetails), 500, 5 * 1000);
|
||||||
|
Assert.assertTrue(
|
||||||
|
isContainerClosed(cluster, containerID, datanodeDetails));
|
||||||
|
}
|
||||||
|
|
||||||
private Boolean isContainerClosed(MiniOzoneCluster ozoneCluster,
|
private Boolean isContainerClosed(MiniOzoneCluster ozoneCluster,
|
||||||
long containerID,
|
long containerID,
|
||||||
DatanodeDetails datanode) {
|
DatanodeDetails datanode) {
|
||||||
|
|
|
@ -27,6 +27,7 @@ import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
|
@ -41,10 +42,12 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.ozone.RatisTestHelper;
|
import org.apache.hadoop.ozone.RatisTestHelper;
|
||||||
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
||||||
|
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
|
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
|
||||||
import org.apache.hadoop.ozone.container.common.transport.server
|
import org.apache.hadoop.ozone.container.common.transport.server
|
||||||
.XceiverServerSpi;
|
.XceiverServerSpi;
|
||||||
|
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
|
||||||
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
@ -185,6 +188,7 @@ public class TestCSMMetrics {
|
||||||
|
|
||||||
final ContainerDispatcher dispatcher = new TestContainerDispatcher();
|
final ContainerDispatcher dispatcher = new TestContainerDispatcher();
|
||||||
return XceiverServerRatis.newXceiverServerRatis(dn, conf, dispatcher,
|
return XceiverServerRatis.newXceiverServerRatis(dn, conf, dispatcher,
|
||||||
|
new ContainerController(new ContainerSet(), Maps.newHashMap()),
|
||||||
null, null);
|
null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -141,8 +141,9 @@ public class TestContainerServer {
|
||||||
conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
|
conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
|
||||||
|
|
||||||
final ContainerDispatcher dispatcher = new TestContainerDispatcher();
|
final ContainerDispatcher dispatcher = new TestContainerDispatcher();
|
||||||
return XceiverServerRatis
|
return XceiverServerRatis.newXceiverServerRatis(dn, conf, dispatcher,
|
||||||
.newXceiverServerRatis(dn, conf, dispatcher, null, caClient);
|
new ContainerController(new ContainerSet(), Maps.newHashMap()),
|
||||||
|
caClient, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
static void runTestClientServerRatis(RpcType rpc, int numNodes)
|
static void runTestClientServerRatis(RpcType rpc, int numNodes)
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.container.server;
|
package org.apache.hadoop.ozone.container.server;
|
||||||
|
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import org.apache.commons.lang3.RandomUtils;
|
import org.apache.commons.lang3.RandomUtils;
|
||||||
import org.apache.commons.lang3.exception.ExceptionUtils;
|
import org.apache.commons.lang3.exception.ExceptionUtils;
|
||||||
import org.apache.hadoop.hdds.HddsConfigKeys;
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
|
@ -149,8 +150,9 @@ public class TestSecureContainerServer {
|
||||||
conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
|
conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
|
||||||
|
|
||||||
final ContainerDispatcher dispatcher = new TestContainerDispatcher();
|
final ContainerDispatcher dispatcher = new TestContainerDispatcher();
|
||||||
return XceiverServerRatis
|
return XceiverServerRatis.newXceiverServerRatis(dn, conf, dispatcher,
|
||||||
.newXceiverServerRatis(dn, conf, dispatcher, null, caClient);
|
new ContainerController(new ContainerSet(), Maps.newHashMap()),
|
||||||
|
caClient, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
static void runTestClientServerRatis(RpcType rpc, int numNodes)
|
static void runTestClientServerRatis(RpcType rpc, int numNodes)
|
||||||
|
|
|
@ -489,7 +489,7 @@ public final class OzoneManagerRatisServer {
|
||||||
.getDuration(), nodeFailureTimeoutUnit);
|
.getDuration(), nodeFailureTimeoutUnit);
|
||||||
final TimeDuration nodeFailureTimeout = TimeDuration.valueOf(
|
final TimeDuration nodeFailureTimeout = TimeDuration.valueOf(
|
||||||
nodeFailureTimeoutDuration, nodeFailureTimeoutUnit);
|
nodeFailureTimeoutDuration, nodeFailureTimeoutUnit);
|
||||||
RaftServerConfigKeys.setLeaderElectionTimeout(properties,
|
RaftServerConfigKeys.Notification.setNoLeaderTimeout(properties,
|
||||||
nodeFailureTimeout);
|
nodeFailureTimeout);
|
||||||
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
|
RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
|
||||||
nodeFailureTimeout);
|
nodeFailureTimeout);
|
||||||
|
|
|
@ -29,7 +29,7 @@
|
||||||
<hadoop.version>3.2.0</hadoop.version>
|
<hadoop.version>3.2.0</hadoop.version>
|
||||||
<hdds.version>0.5.0-SNAPSHOT</hdds.version>
|
<hdds.version>0.5.0-SNAPSHOT</hdds.version>
|
||||||
<ozone.version>0.5.0-SNAPSHOT</ozone.version>
|
<ozone.version>0.5.0-SNAPSHOT</ozone.version>
|
||||||
<ratis.version>0.4.0-2337318-SNAPSHOT</ratis.version>
|
<ratis.version>0.4.0-78e95b9-SNAPSHOT</ratis.version>
|
||||||
<bouncycastle.version>1.60</bouncycastle.version>
|
<bouncycastle.version>1.60</bouncycastle.version>
|
||||||
<ozone.release>Crater Lake</ozone.release>
|
<ozone.release>Crater Lake</ozone.release>
|
||||||
<declared.ozone.version>${ozone.version}</declared.ozone.version>
|
<declared.ozone.version>${ozone.version}</declared.ozone.version>
|
||||||
|
|
Loading…
Reference in New Issue