HDDS-801. Quasi close the container when close is not executed via Ratis.

Contributed by Nanda kumar.
This commit is contained in:
Nanda kumar 2018-11-16 23:07:45 +05:30
parent ad5256e44d
commit c4d9640028
38 changed files with 876 additions and 369 deletions

View File

@ -229,8 +229,8 @@ message ContainerDataProto {
enum State { enum State {
OPEN = 1; OPEN = 1;
CLOSING = 2; CLOSING = 2;
CLOSED = 3; QUASI_CLOSED =3;
QUASI_CLOSED = 4; CLOSED = 4;
UNHEALTHY = 5; UNHEALTHY = 5;
INVALID = 6; INVALID = 6;
} }

View File

@ -241,6 +241,21 @@ public synchronized boolean isClosed() {
return ContainerDataProto.State.CLOSED == state; return ContainerDataProto.State.CLOSED == state;
} }
/**
* checks if the container is quasi closed.
* @return - boolean
*/
public synchronized boolean isQuasiClosed() {
return ContainerDataProto.State.QUASI_CLOSED == state;
}
/**
* Marks this container as quasi closed.
*/
public synchronized void quasiCloseContainer() {
setState(ContainerDataProto.State.QUASI_CLOSED);
}
/** /**
* Marks this container as closed. * Marks this container as closed.
*/ */

View File

@ -20,7 +20,6 @@
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@ -76,18 +75,14 @@ public class HddsDispatcher implements ContainerDispatcher {
* XceiverServerHandler. * XceiverServerHandler.
*/ */
public HddsDispatcher(Configuration config, ContainerSet contSet, public HddsDispatcher(Configuration config, ContainerSet contSet,
VolumeSet volumes, StateContext context) { VolumeSet volumes, Map<ContainerType, Handler> handlers,
StateContext context, ContainerMetrics metrics) {
this.conf = config; this.conf = config;
this.containerSet = contSet; this.containerSet = contSet;
this.volumeSet = volumes; this.volumeSet = volumes;
this.context = context; this.context = context;
this.handlers = Maps.newHashMap(); this.handlers = handlers;
this.metrics = ContainerMetrics.create(conf); this.metrics = metrics;
for (ContainerType containerType : ContainerType.values()) {
handlers.put(containerType,
Handler.getHandlerForContainerType(
containerType, conf, containerSet, volumeSet, metrics));
}
this.containerCloseThreshold = conf.getFloat( this.containerCloseThreshold = conf.getFloat(
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD, HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD,
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT); HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
@ -347,6 +342,7 @@ public void setScmId(String scmId) {
} }
} }
@VisibleForTesting
public Container getContainer(long containerID) { public Container getContainer(long containerID) {
return containerSet.getContainer(containerID); return containerSet.getContainer(containerID);
} }

View File

@ -82,11 +82,24 @@ void update(Map<String, String> metaData, boolean forceUpdate)
ContainerProtos.ContainerDataProto.State getContainerState(); ContainerProtos.ContainerDataProto.State getContainerState();
/** /**
* Closes a open container, if it is already closed or does not exist a * Marks the container for closing. Moves the container to CLOSING state.
*/
void markContainerForClose() throws StorageContainerException;
/**
* Quasi Closes a open container, if it is already closed or does not exist a
* StorageContainerException is thrown. * StorageContainerException is thrown.
* *
* @throws StorageContainerException * @throws StorageContainerException
*/ */
void quasiClose() throws StorageContainerException;
/**
* Closes a open/quasi closed container, if it is already closed or does not
* exist a StorageContainerException is thrown.
*
* @throws StorageContainerException
*/
void close() throws StorageContainerException; void close() throws StorageContainerException;
/** /**

View File

@ -29,8 +29,12 @@
.ContainerCommandResponseProto; .ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerType; .ContainerType;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.IncrementalContainerReportProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker; import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
@ -47,26 +51,47 @@ public abstract class Handler {
protected String scmID; protected String scmID;
protected final ContainerMetrics metrics; protected final ContainerMetrics metrics;
protected Handler(Configuration config, ContainerSet contSet, private final StateContext context;
VolumeSet volumeSet, ContainerMetrics containerMetrics) {
conf = config; protected Handler(Configuration config, StateContext context,
containerSet = contSet; ContainerSet contSet, VolumeSet volumeSet,
ContainerMetrics containerMetrics) {
this.conf = config;
this.context = context;
this.containerSet = contSet;
this.volumeSet = volumeSet; this.volumeSet = volumeSet;
this.metrics = containerMetrics; this.metrics = containerMetrics;
} }
public static Handler getHandlerForContainerType(ContainerType containerType, public static Handler getHandlerForContainerType(
Configuration config, ContainerSet contSet, VolumeSet volumeSet, final ContainerType containerType, final Configuration config,
ContainerMetrics metrics) { final StateContext context, final ContainerSet contSet,
final VolumeSet volumeSet, final ContainerMetrics metrics) {
switch (containerType) { switch (containerType) {
case KeyValueContainer: case KeyValueContainer:
return new KeyValueHandler(config, contSet, volumeSet, metrics); return new KeyValueHandler(config, context, contSet, volumeSet, metrics);
default: default:
throw new IllegalArgumentException("Handler for ContainerType: " + throw new IllegalArgumentException("Handler for ContainerType: " +
containerType + "doesn't exist."); containerType + "doesn't exist.");
} }
} }
/**
* This should be called whenever there is state change. It will trigger
* an ICR to SCM.
*
* @param container Container for which ICR has to be sent
*/
protected void sendICR(final Container container)
throws StorageContainerException {
IncrementalContainerReportProto icr = IncrementalContainerReportProto
.newBuilder()
.addReport(container.getContainerReport())
.build();
context.addReport(icr);
context.getParent().triggerHeartbeat();
}
public abstract ContainerCommandResponseProto handle( public abstract ContainerCommandResponseProto handle(
ContainerCommandRequestProto msg, Container container); ContainerCommandRequestProto msg, Container container);
@ -80,6 +105,33 @@ public abstract Container importContainer(
TarContainerPacker packer) TarContainerPacker packer)
throws IOException; throws IOException;
/**
* Marks the container for closing. Moves the container to CLOSING state.
*
* @param container container to update
* @throws IOException in case of exception
*/
public abstract void markContainerForClose(Container container)
throws IOException;
/**
* Moves the Container to QUASI_CLOSED state.
*
* @param container container to be quasi closed
* @throws IOException
*/
public abstract void quasiCloseContainer(Container container)
throws IOException;
/**
* Moves the Container to CLOSED state.
*
* @param container container to be closed
* @throws IOException
*/
public abstract void closeContainer(Container container)
throws IOException;
public void setScmID(String scmId) { public void setScmID(String scmId) {
this.scmID = scmId; this.scmID = scmId;
} }

View File

@ -80,6 +80,7 @@ private long getRandomReportDelay() {
@Override @Override
protected ContainerReportsProto getReport() throws IOException { protected ContainerReportsProto getReport() throws IOException {
return getContext().getParent().getContainer().getContainerReport(); return getContext().getParent().getContainer()
.getController().getContainerReport();
} }
} }

View File

@ -99,7 +99,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
ContainerReplicator replicator = ContainerReplicator replicator =
new DownloadAndImportReplicator(container.getContainerSet(), new DownloadAndImportReplicator(container.getContainerSet(),
container.getDispatcher(), container.getController(),
new SimpleContainerDownloader(conf), new TarContainerPacker()); new SimpleContainerDownloader(conf), new TarContainerPacker());
supervisor = supervisor =

View File

@ -59,8 +59,6 @@
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Consumer; import java.util.function.Consumer;
import static org.apache.hadoop.ozone.OzoneConsts.INVALID_PORT;
/** /**
* Current Context of State Machine. * Current Context of State Machine.
*/ */
@ -115,24 +113,6 @@ public DatanodeStateMachine getParent() {
return parent; return parent;
} }
/**
* Get the container server port.
* @return The container server port if available, return -1 if otherwise
*/
public int getContainerPort() {
return parent == null ?
INVALID_PORT : parent.getContainer().getContainerServerPort();
}
/**
* Gets the Ratis Port.
* @return int , return -1 if not valid.
*/
public int getRatisPort() {
return parent == null ?
INVALID_PORT : parent.getContainer().getRatisContainerServerPort();
}
/** /**
* Returns true if we are entering a new state. * Returns true if we are entering a new state.
* *

View File

@ -16,21 +16,20 @@
*/ */
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import com.google.protobuf.InvalidProtocolBufferException;
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;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.datanode.proto
import org.apache.hadoop.hdds.protocol.proto .ContainerProtos.ContainerCommandRequestProto;
.StorageContainerDatanodeProtocolProtos.IncrementalContainerReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto; .StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
ContainerDataProto.State;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.common.statemachine import org.apache.hadoop.ozone.container.common.statemachine
.SCMConnectionManager; .SCMConnectionManager;
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.ContainerController;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
@ -38,17 +37,19 @@
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.UUID; import java.util.UUID;
/** /**
* Handler for close container command received from SCM. * Handler for close container command received from SCM.
*/ */
public class CloseContainerCommandHandler implements CommandHandler { public class CloseContainerCommandHandler implements CommandHandler {
static final Logger LOG =
private static final Logger LOG =
LoggerFactory.getLogger(CloseContainerCommandHandler.class); LoggerFactory.getLogger(CloseContainerCommandHandler.class);
private int invocationCount; private int invocationCount;
private long totalTime; private long totalTime;
private boolean cmdExecuted;
/** /**
* Constructs a ContainerReport handler. * Constructs a ContainerReport handler.
@ -67,77 +68,68 @@ public CloseContainerCommandHandler() {
@Override @Override
public void handle(SCMCommand command, OzoneContainer ozoneContainer, public void handle(SCMCommand command, OzoneContainer ozoneContainer,
StateContext context, SCMConnectionManager connectionManager) { StateContext context, SCMConnectionManager connectionManager) {
try {
LOG.debug("Processing Close Container command."); LOG.debug("Processing Close Container command.");
invocationCount++; invocationCount++;
long startTime = Time.monotonicNow(); final long startTime = Time.monotonicNow();
// TODO: define this as INVALID_CONTAINER_ID in HddsConsts.java (TBA) final DatanodeDetails datanodeDetails = context.getParent()
long containerID = -1; .getDatanodeDetails();
try { final CloseContainerCommandProto closeCommand =
CloseContainerCommandProto closeContainerProto =
CloseContainerCommandProto.parseFrom(command.getProtoBufMessage()); CloseContainerCommandProto.parseFrom(command.getProtoBufMessage());
containerID = closeContainerProto.getContainerID(); final ContainerController controller = ozoneContainer.getController();
// CloseContainer operation is idempotent, if the container is already final long containerId = closeCommand.getContainerID();
// closed, then do nothing. try {
// TODO: Non-existent container should be handled properly // TODO: Closing of QUASI_CLOSED container.
Container container =
ozoneContainer.getContainerSet().getContainer(containerID); final Container container = controller.getContainer(containerId);
if (container == null) { if (container == null) {
LOG.error("Container {} does not exist in datanode. " LOG.error("Container #{} does not exist in datanode. "
+ "Container close failed.", containerID); + "Container close failed.", containerId);
cmdExecuted = false;
return; return;
} }
ContainerData containerData = container.getContainerData();
State containerState = container.getContainerData().getState(); // Move the container to CLOSING state
if (containerState != State.CLOSED) { controller.markContainerForClose(containerId);
LOG.debug("Closing container {}.", containerID);
// when a closeContainerCommand arrives at a Datanode and if the // If the container is part of open pipeline, close it via write channel
// container is open, each replica will be moved to closing state first. if (ozoneContainer.getWriteChannel()
if (containerState == State.OPEN) { .isExist(closeCommand.getPipelineID())) {
containerData.setState(State.CLOSING); ContainerCommandRequestProto request =
getContainerCommandRequestProto(datanodeDetails,
closeCommand.getContainerID());
ozoneContainer.getWriteChannel().submitRequest(
request, closeCommand.getPipelineID());
return;
} }
// if the container is already closed, it will be just ignored. // The container is not part of any open pipeline.
// ICR will get triggered to change the replica state in SCM. // QUASI_CLOSE the container using ContainerController.
HddsProtos.PipelineID pipelineID = closeContainerProto.getPipelineID(); controller.quasiCloseContainer(containerId);
HddsProtos.ReplicationType replicationType = } catch (NotLeaderException e) {
closeContainerProto.getReplicationType(); LOG.debug("Follower cannot close container #{}.", containerId);
} catch (IOException e) {
ContainerProtos.ContainerCommandRequestProto.Builder request = LOG.error("Can't close container #{}", containerId, e);
ContainerProtos.ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.CloseContainer);
request.setContainerID(containerID);
request.setCloseContainer(
ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(
context.getParent().getDatanodeDetails().getUuidString());
// submit the close container request for the XceiverServer to handle
ozoneContainer.submitContainerRequest(request.build(), replicationType,
pipelineID);
// Since the container is closed, we trigger an ICR
IncrementalContainerReportProto icr =
IncrementalContainerReportProto.newBuilder().addReport(
ozoneContainer.getContainerSet().getContainer(containerID)
.getContainerReport()).build();
context.addReport(icr);
context.getParent().triggerHeartbeat();
}
} catch (Exception e) {
if (e instanceof NotLeaderException) {
// If the particular datanode is not the Ratis leader, the close
// container command will not be executed by the follower but will be
// executed by Ratis stateMachine transactions via leader to follower.
// There can also be case where the datanode is in candidate state.
// In these situations, NotLeaderException is thrown.
LOG.info("Follower cannot close the container {}.", containerID);
} else {
LOG.error("Can't close container " + containerID, e);
}
} finally { } finally {
long endTime = Time.monotonicNow(); long endTime = Time.monotonicNow();
totalTime += endTime - startTime; totalTime += endTime - startTime;
} }
} catch (InvalidProtocolBufferException ex) {
LOG.error("Exception while closing container", ex);
}
}
private ContainerCommandRequestProto getContainerCommandRequestProto(
final DatanodeDetails datanodeDetails, final long containerId) {
final ContainerCommandRequestProto.Builder command =
ContainerCommandRequestProto.newBuilder();
command.setCmdType(ContainerProtos.Type.CloseContainer);
command.setContainerID(containerId);
command.setCloseContainer(
ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
command.setTraceID(UUID.randomUUID().toString());
command.setDatanodeUuid(datanodeDetails.getUuidString());
return command.build();
} }
/** /**

View File

@ -111,7 +111,7 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
try { try {
ContainerReportsProto containerReport = datanodeContainerManager ContainerReportsProto containerReport = datanodeContainerManager
.getContainerReport(); .getController().getContainerReport();
NodeReportProto nodeReport = datanodeContainerManager.getNodeReport(); NodeReportProto nodeReport = datanodeContainerManager.getNodeReport();
PipelineReportsProto pipelineReportsProto = PipelineReportsProto pipelineReportsProto =
datanodeContainerManager.getPipelineReport(); datanodeContainerManager.getPipelineReport();

View File

@ -140,6 +140,11 @@ public void submitRequest(ContainerCommandRequestProto request,
} }
} }
@Override
public boolean isExist(HddsProtos.PipelineID pipelineId) {
return PipelineID.valueOf(id).getProtobuf().equals(pipelineId);
}
@Override @Override
public List<PipelineReport> getPipelineReport() { public List<PipelineReport> getPipelineReport() {
return Collections.singletonList( return Collections.singletonList(

View File

@ -53,6 +53,13 @@ void submitRequest(ContainerCommandRequestProto request,
HddsProtos.PipelineID pipelineID) HddsProtos.PipelineID pipelineID)
throws IOException; throws IOException;
/**
* Returns true if the given pipeline exist.
*
* @return true if pipeline present, else false
*/
boolean isExist(HddsProtos.PipelineID pipelineId);
/** /**
* Get pipeline report for the XceiverServer instance. * Get pipeline report for the XceiverServer instance.
* @return list of report for each pipeline. * @return list of report for each pipeline.

View File

@ -86,7 +86,8 @@
* Ozone containers. * Ozone containers.
*/ */
public final class XceiverServerRatis implements XceiverServerSpi { public final class XceiverServerRatis implements XceiverServerSpi {
static final Logger LOG = LoggerFactory.getLogger(XceiverServerRatis.class); private static final Logger LOG = LoggerFactory
.getLogger(XceiverServerRatis.class);
private static final AtomicLong CALL_ID_COUNTER = new AtomicLong(); private static final AtomicLong CALL_ID_COUNTER = new AtomicLong();
private static long nextCallId() { private static long nextCallId() {
@ -457,6 +458,21 @@ private void handlePipelineFailure(RaftGroupId groupId,
+ ".Reason : " + action.getClosePipeline().getDetailedReason()); + ".Reason : " + action.getClosePipeline().getDetailedReason());
} }
@Override
public boolean isExist(HddsProtos.PipelineID pipelineId) {
try {
for (RaftGroupId groupId : server.getGroupIds()) {
if (PipelineID.valueOf(
groupId.getUuid()).getProtobuf().equals(pipelineId)) {
return true;
}
}
return false;
} catch (IOException e) {
return false;
}
}
@Override @Override
public List<PipelineReport> getPipelineReport() { public List<PipelineReport> getPipelineReport() {
try { try {

View File

@ -31,6 +31,8 @@
import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerDataProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerType; .ContainerType;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -267,29 +269,47 @@ public void delete(boolean forceDelete)
} }
@Override @Override
public void close() throws StorageContainerException { public void markContainerForClose() throws StorageContainerException {
updateContainerData(() ->
containerData.setState(ContainerDataProto.State.CLOSING));
}
//TODO: writing .container file and compaction can be done @Override
// asynchronously, otherwise rpc call for this will take a lot of time to public void quasiClose() throws StorageContainerException {
// complete this action updateContainerData(containerData::quasiCloseContainer);
}
@Override
public void close() throws StorageContainerException {
updateContainerData(containerData::closeContainer);
// It is ok if this operation takes a bit of time.
// Close container is not expected to be instantaneous.
compactDB();
}
private void updateContainerData(Runnable update)
throws StorageContainerException {
ContainerDataProto.State oldState = null;
try { try {
writeLock(); writeLock();
oldState = containerData.getState();
containerData.closeContainer(); update.run();
File containerFile = getContainerFile(); File containerFile = getContainerFile();
// update the new container data to .container File // update the new container data to .container File
updateContainerFile(containerFile); updateContainerFile(containerFile);
} catch (StorageContainerException ex) { } catch (StorageContainerException ex) {
if (oldState != null) {
// Failed to update .container file. Reset the state to CLOSING // Failed to update .container file. Reset the state to CLOSING
containerData.setState(ContainerProtos.ContainerDataProto.State.CLOSING); containerData.setState(oldState);
}
throw ex; throw ex;
} finally { } finally {
writeUnlock(); writeUnlock();
} }
}
// It is ok if this operation takes a bit of time. private void compactDB() throws StorageContainerException {
// Close container is not expected to be instantaneous.
try { try {
MetadataStore db = BlockUtils.getDB(containerData, config); MetadataStore db = BlockUtils.getDB(containerData, config);
db.compactDB(); db.compactDB();
@ -549,6 +569,9 @@ private ContainerReplicaProto.State getHddsState()
case CLOSING: case CLOSING:
state = ContainerReplicaProto.State.CLOSING; state = ContainerReplicaProto.State.CLOSING;
break; break;
case QUASI_CLOSED:
state = ContainerReplicaProto.State.QUASI_CLOSED;
break;
case CLOSED: case CLOSED:
state = ContainerReplicaProto.State.CLOSED; state = ContainerReplicaProto.State.CLOSED;
break; break;

View File

@ -33,7 +33,7 @@
import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerDataProto; .ContainerDataProto.State;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto; .ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@ -57,6 +57,7 @@
import org.apache.hadoop.ozone.container.common.interfaces.Container; 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.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy; import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.apache.hadoop.ozone.container.common.volume import org.apache.hadoop.ozone.container.common.volume
.RoundRobinVolumeChoosingPolicy; .RoundRobinVolumeChoosingPolicy;
@ -109,9 +110,9 @@ public class KeyValueHandler extends Handler {
private final long maxContainerSize; private final long maxContainerSize;
private final AutoCloseableLock handlerLock; private final AutoCloseableLock handlerLock;
public KeyValueHandler(Configuration config, ContainerSet contSet, public KeyValueHandler(Configuration config, StateContext context,
VolumeSet volSet, ContainerMetrics metrics) { ContainerSet contSet, VolumeSet volSet, ContainerMetrics metrics) {
super(config, contSet, volSet, metrics); super(config, context, contSet, volSet, metrics);
containerType = ContainerType.KeyValueContainer; containerType = ContainerType.KeyValueContainer;
blockManager = new BlockManagerImpl(config); blockManager = new BlockManagerImpl(config);
chunkManager = new ChunkManagerImpl(); chunkManager = new ChunkManagerImpl();
@ -372,20 +373,10 @@ ContainerCommandResponseProto handleCloseContainer(
request.getTraceID()); request.getTraceID());
return ContainerUtils.malformedRequest(request); return ContainerUtils.malformedRequest(request);
} }
long containerID = kvContainer.getContainerData().getContainerID();
try { try {
checkContainerOpen(kvContainer); markContainerForClose(kvContainer);
// TODO : The close command should move the container to either quasi closeContainer(kvContainer);
// closed/closed depending upon how the closeContainer gets executed.
// If it arrives by Standalone, it will be moved to Quasi Closed or
// otherwise moved to Closed state if it gets executed via Ratis.
kvContainer.close();
} catch (StorageContainerException ex) { } catch (StorageContainerException ex) {
if (ex.getResult() == CLOSED_CONTAINER_IO) {
LOG.debug("Container {} is already closed.", containerID);
return ContainerUtils.getSuccessResponse(request);
}
return ContainerUtils.logAndReturnError(LOG, ex, request); return ContainerUtils.logAndReturnError(LOG, ex, request);
} catch (IOException ex) { } catch (IOException ex) {
return ContainerUtils.logAndReturnError(LOG, return ContainerUtils.logAndReturnError(LOG,
@ -745,23 +736,24 @@ ContainerCommandResponseProto handleUnsupportedOp(
private void checkContainerOpen(KeyValueContainer kvContainer) private void checkContainerOpen(KeyValueContainer kvContainer)
throws StorageContainerException { throws StorageContainerException {
ContainerDataProto.State containerState = kvContainer.getContainerState(); final State containerState = kvContainer.getContainerState();
/** /*
* In a closing state, follower will receive transactions from leader. * In a closing state, follower will receive transactions from leader.
* Once the leader is put to closing state, it will reject further requests * Once the leader is put to closing state, it will reject further requests
* from clients. Only the transactions which happened before the container * from clients. Only the transactions which happened before the container
* in the leader goes to closing state, will arrive here even the container * in the leader goes to closing state, will arrive here even the container
* might already be in closing state here. * might already be in closing state here.
*/ */
if (containerState == ContainerDataProto.State.OPEN if (containerState == State.OPEN || containerState == State.CLOSING) {
|| containerState == ContainerDataProto.State.CLOSING) {
return; return;
} else { }
String msg = "Requested operation not allowed as ContainerState is " +
containerState; final ContainerProtos.Result result;
ContainerProtos.Result result = null;
switch (containerState) { switch (containerState) {
case QUASI_CLOSED:
result = CLOSED_CONTAINER_IO;
break;
case CLOSED: case CLOSED:
result = CLOSED_CONTAINER_IO; result = CLOSED_CONTAINER_IO;
break; break;
@ -774,10 +766,10 @@ private void checkContainerOpen(KeyValueContainer kvContainer)
default: default:
result = CONTAINER_INTERNAL_ERROR; result = CONTAINER_INTERNAL_ERROR;
} }
String msg = "Requested operation not allowed as ContainerState is " +
containerState;
throw new StorageContainerException(msg, result); throw new StorageContainerException(msg, result);
} }
}
public Container importContainer(long containerID, long maxSize, public Container importContainer(long containerID, long maxSize,
FileInputStream rawContainerStream, FileInputStream rawContainerStream,
@ -796,4 +788,55 @@ public Container importContainer(long containerID, long maxSize,
return container; return container;
} }
@Override
public void markContainerForClose(Container container)
throws IOException {
State currentState = container.getContainerState();
// Move the container to CLOSING state only if it's OPEN
if (currentState == State.OPEN) {
container.markContainerForClose();
sendICR(container);
}
}
@Override
public void quasiCloseContainer(Container container)
throws IOException {
final State state = container.getContainerState();
// Quasi close call is idempotent.
if (state == State.QUASI_CLOSED) {
return;
}
// The container has to be in CLOSING state.
if (state != State.CLOSING) {
ContainerProtos.Result error = state == State.INVALID ?
INVALID_CONTAINER_STATE : CONTAINER_INTERNAL_ERROR;
throw new StorageContainerException("Cannot quasi close container #" +
container.getContainerData().getContainerID() + " while in " +
state + " state.", error);
}
container.quasiClose();
sendICR(container);
}
@Override
public void closeContainer(Container container)
throws IOException {
final State state = container.getContainerState();
// Close call is idempotent.
if (state == State.CLOSED) {
return;
}
// The container has to be either in CLOSING or in QUASI_CLOSED state.
if (state != State.CLOSING && state != State.QUASI_CLOSED) {
ContainerProtos.Result error = state == State.INVALID ?
INVALID_CONTAINER_STATE : CONTAINER_INTERNAL_ERROR;
throw new StorageContainerException("Cannot close container #" +
container.getContainerData().getContainerID() + " while in " +
state + " state.", error);
}
container.close();
sendICR(container);
}
} }

View File

@ -0,0 +1,124 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.ozoneimpl;
import org.apache.hadoop.hdds.protocol.datanode.proto
.ContainerProtos.ContainerType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerDataProto.State;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
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.keyvalue.TarContainerPacker;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.Map;
/**
* Control plane for container management in datanode.
*/
public class ContainerController {
private final ContainerSet containerSet;
private final Map<ContainerType, Handler> handlers;
public ContainerController(final ContainerSet containerSet,
final Map<ContainerType, Handler> handlers) {
this.containerSet = containerSet;
this.handlers = handlers;
}
/**
* Returns the Container given a container id.
*
* @param containerId ID of the container
* @return Container
*/
public Container getContainer(final long containerId) {
return containerSet.getContainer(containerId);
}
/**
* Marks the container for closing. Moves the container to CLOSING state.
*
* @param containerId Id of the container to update
* @throws IOException in case of exception
*/
public void markContainerForClose(final long containerId)
throws IOException {
Container container = containerSet.getContainer(containerId);
if (container.getContainerState() == State.OPEN) {
getHandler(container).markContainerForClose(container);
}
}
/**
* Returns the container report.
*
* @return ContainerReportsProto
* @throws IOException in case of exception
*/
public ContainerReportsProto getContainerReport()
throws IOException {
return containerSet.getContainerReport();
}
/**
* Quasi closes a container given its id.
*
* @param containerId Id of the container to quasi close
* @throws IOException in case of exception
*/
public void quasiCloseContainer(final long containerId) throws IOException {
final Container container = containerSet.getContainer(containerId);
getHandler(container).quasiCloseContainer(container);
}
/**
* Closes a container given its id.
*
* @param containerId Id of the container to close
* @throws IOException in case of exception
*/
public void closeContainer(final long containerId) throws IOException {
final Container container = containerSet.getContainer(containerId);
getHandler(container).closeContainer(container);
}
public Container importContainer(final ContainerType type,
final long containerId, final long maxSize,
final FileInputStream rawContainerStream, final TarContainerPacker packer)
throws IOException {
return handlers.get(type).importContainer(
containerId, maxSize, rawContainerStream, packer);
}
/**
* Given a container, returns its handler instance.
*
* @param container Container
* @return handler of the container
*/
private Handler getHandler(final Container container) {
return handlers.get(container.getContainerType());
}
}

View File

@ -19,18 +19,20 @@
package org.apache.hadoop.ozone.container.ozoneimpl; package org.apache.hadoop.ozone.container.ozoneimpl;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.collect.Maps;
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;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.PipelineID; .ContainerProtos.ContainerType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; .StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
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.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
@ -47,27 +49,26 @@
import java.io.*; import java.io.*;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.Map; import java.util.Map;
import static org.apache.hadoop.ozone.OzoneConsts.INVALID_PORT;
/** /**
* Ozone main class sets up the network servers and initializes the container * Ozone main class sets up the network servers and initializes the container
* layer. * layer.
*/ */
public class OzoneContainer { public class OzoneContainer {
public static final Logger LOG = LoggerFactory.getLogger( private static final Logger LOG = LoggerFactory.getLogger(
OzoneContainer.class); OzoneContainer.class);
private final HddsDispatcher hddsDispatcher; private final HddsDispatcher hddsDispatcher;
private final DatanodeDetails dnDetails; private final Map<ContainerType, Handler> handlers;
private final OzoneConfiguration config; private final OzoneConfiguration config;
private final VolumeSet volumeSet; private final VolumeSet volumeSet;
private final ContainerSet containerSet; private final ContainerSet containerSet;
private final Map<ReplicationType, XceiverServerSpi> servers; private final XceiverServerSpi writeChannel;
private final XceiverServerSpi readChannel;
private final ContainerController controller;
/** /**
* Construct OzoneContainer object. * Construct OzoneContainer object.
@ -78,31 +79,42 @@ public class OzoneContainer {
*/ */
public OzoneContainer(DatanodeDetails datanodeDetails, OzoneConfiguration public OzoneContainer(DatanodeDetails datanodeDetails, OzoneConfiguration
conf, StateContext context) throws IOException { conf, StateContext context) throws IOException {
this.dnDetails = datanodeDetails;
this.config = conf; this.config = conf;
this.volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf); this.volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
this.containerSet = new ContainerSet(); this.containerSet = new ContainerSet();
buildContainerSet(); buildContainerSet();
hddsDispatcher = new HddsDispatcher(config, containerSet, volumeSet, final ContainerMetrics metrics = ContainerMetrics.create(conf);
context); this.handlers = Maps.newHashMap();
servers = new HashMap<>(); for (ContainerType containerType : ContainerType.values()) {
servers.put(ReplicationType.STAND_ALONE, handlers.put(containerType,
new XceiverServerGrpc(datanodeDetails, config, hddsDispatcher, Handler.getHandlerForContainerType(
createReplicationService())); containerType, conf, context, containerSet, volumeSet, metrics));
servers.put(ReplicationType.RATIS, XceiverServerRatis }
.newXceiverServerRatis(datanodeDetails, config, hddsDispatcher, this.hddsDispatcher = new HddsDispatcher(config, containerSet, volumeSet,
context)); handlers, context, metrics);
/*
* ContainerController is the control plane
* XceiverServerRatis is the write channel
* XceiverServerGrpc is the read channel
*/
this.controller = new ContainerController(containerSet, handlers);
this.writeChannel = XceiverServerRatis.newXceiverServerRatis(
datanodeDetails, config, hddsDispatcher, context);
this.readChannel = new XceiverServerGrpc(
datanodeDetails, config, hddsDispatcher, createReplicationService());
} }
private GrpcReplicationService createReplicationService() { private GrpcReplicationService createReplicationService() {
return new GrpcReplicationService( return new GrpcReplicationService(
new OnDemandContainerReplicationSource(containerSet)); new OnDemandContainerReplicationSource(controller));
} }
/** /**
* Build's container map. * Build's container map.
*/ */
public void buildContainerSet() { private void buildContainerSet() {
Iterator<HddsVolume> volumeSetIterator = volumeSet.getVolumesList() Iterator<HddsVolume> volumeSetIterator = volumeSet.getVolumesList()
.iterator(); .iterator();
ArrayList<Thread> volumeThreads = new ArrayList<Thread>(); ArrayList<Thread> volumeThreads = new ArrayList<Thread>();
@ -111,7 +123,6 @@ public void buildContainerSet() {
// And also handle disk failure tolerance need to be added // And also handle disk failure tolerance need to be added
while (volumeSetIterator.hasNext()) { while (volumeSetIterator.hasNext()) {
HddsVolume volume = volumeSetIterator.next(); HddsVolume volume = volumeSetIterator.next();
File hddsVolumeRootDir = volume.getHddsRootDir();
Thread thread = new Thread(new ContainerReader(volumeSet, volume, Thread thread = new Thread(new ContainerReader(volumeSet, volume,
containerSet, config)); containerSet, config));
thread.start(); thread.start();
@ -135,9 +146,8 @@ public void buildContainerSet() {
*/ */
public void start() throws IOException { public void start() throws IOException {
LOG.info("Attempting to start container services."); LOG.info("Attempting to start container services.");
for (XceiverServerSpi serverinstance : servers.values()) { writeChannel.start();
serverinstance.start(); readChannel.start();
}
hddsDispatcher.init(); hddsDispatcher.init();
} }
@ -147,9 +157,8 @@ public void start() throws IOException {
public void stop() { public void stop() {
//TODO: at end of container IO integration work. //TODO: at end of container IO integration work.
LOG.info("Attempting to stop container services."); LOG.info("Attempting to stop container services.");
for(XceiverServerSpi serverinstance: servers.values()) { writeChannel.stop();
serverinstance.stop(); readChannel.stop();
}
hddsDispatcher.shutdown(); hddsDispatcher.shutdown();
} }
@ -163,58 +172,24 @@ public ContainerSet getContainerSet() {
* @return - container report. * @return - container report.
* @throws IOException * @throws IOException
*/ */
public StorageContainerDatanodeProtocolProtos.ContainerReportsProto
getContainerReport() throws IOException {
return this.containerSet.getContainerReport();
}
public PipelineReportsProto getPipelineReport() { public PipelineReportsProto getPipelineReport() {
PipelineReportsProto.Builder pipelineReportsProto = PipelineReportsProto.Builder pipelineReportsProto =
PipelineReportsProto.newBuilder(); PipelineReportsProto.newBuilder();
for (XceiverServerSpi serverInstance : servers.values()) { pipelineReportsProto.addAllPipelineReport(writeChannel.getPipelineReport());
pipelineReportsProto
.addAllPipelineReport(serverInstance.getPipelineReport());
}
return pipelineReportsProto.build(); return pipelineReportsProto.build();
} }
/** public XceiverServerSpi getWriteChannel() {
* Submit ContainerRequest. return writeChannel;
* @param request
* @param replicationType
* @param pipelineID
*/
public void submitContainerRequest(
ContainerProtos.ContainerCommandRequestProto request,
ReplicationType replicationType,
PipelineID pipelineID) throws IOException {
LOG.info("submitting {} request over {} server for container {}",
request.getCmdType(), replicationType, request.getContainerID());
Preconditions.checkState(servers.containsKey(replicationType));
servers.get(replicationType).submitRequest(request, pipelineID);
} }
private int getPortByType(ReplicationType replicationType) { public XceiverServerSpi getReadChannel() {
return servers.containsKey(replicationType) ? return readChannel;
servers.get(replicationType).getIPCPort() : INVALID_PORT;
} }
/** public ContainerController getController() {
* Returns the container servers IPC port. return controller;
*
* @return Container servers IPC port.
*/
public int getContainerServerPort() {
return getPortByType(ReplicationType.STAND_ALONE);
}
/**
* Returns the Ratis container Server IPC port.
*
* @return Ratis port.
*/
public int getRatisContainerServerPort() {
return getPortByType(ReplicationType.RATIS);
} }
/** /**
@ -230,11 +205,6 @@ public ContainerDispatcher getDispatcher() {
return this.hddsDispatcher; return this.hddsDispatcher;
} }
@VisibleForTesting
public XceiverServerSpi getServer(ReplicationType replicationType) {
return servers.get(replicationType);
}
public VolumeSet getVolumeSet() { public VolumeSet getVolumeSet() {
return volumeSet; return volumeSet;
} }

View File

@ -28,9 +28,8 @@
import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml; import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet; 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.Container;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker; import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.ozone.container.replication.ReplicationTask.Status; import org.apache.hadoop.ozone.container.replication.ReplicationTask.Status;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -49,7 +48,7 @@ public class DownloadAndImportReplicator implements ContainerReplicator {
private final ContainerSet containerSet; private final ContainerSet containerSet;
private final ContainerDispatcher containerDispatcher; private final ContainerController controller;
private final ContainerDownloader downloader; private final ContainerDownloader downloader;
@ -57,11 +56,11 @@ public class DownloadAndImportReplicator implements ContainerReplicator {
public DownloadAndImportReplicator( public DownloadAndImportReplicator(
ContainerSet containerSet, ContainerSet containerSet,
ContainerDispatcher containerDispatcher, ContainerController controller,
ContainerDownloader downloader, ContainerDownloader downloader,
TarContainerPacker packer) { TarContainerPacker packer) {
this.containerSet = containerSet; this.containerSet = containerSet;
this.containerDispatcher = containerDispatcher; this.controller = controller;
this.downloader = downloader; this.downloader = downloader;
this.packer = packer; this.packer = packer;
} }
@ -80,10 +79,9 @@ public void importContainer(long containerID, Path tarFilePath) {
try (FileInputStream tempContainerTarStream = new FileInputStream( try (FileInputStream tempContainerTarStream = new FileInputStream(
tarFilePath.toFile())) { tarFilePath.toFile())) {
Handler handler = containerDispatcher.getHandler( Container container = controller.importContainer(
originalContainerData.getContainerType()); originalContainerData.getContainerType(),
containerID,
Container container = handler.importContainer(containerID,
originalContainerData.getMaxSize(), originalContainerData.getMaxSize(),
tempContainerTarStream, tempContainerTarStream,
packer); packer);

View File

@ -20,12 +20,12 @@
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
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.Container;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker; import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker; import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -39,13 +39,13 @@ public class OnDemandContainerReplicationSource
private static final Logger LOG = private static final Logger LOG =
LoggerFactory.getLogger(ContainerReplicationSource.class); LoggerFactory.getLogger(ContainerReplicationSource.class);
private ContainerSet containerSet; private ContainerController controller;
private ContainerPacker packer = new TarContainerPacker(); private ContainerPacker packer = new TarContainerPacker();
public OnDemandContainerReplicationSource( public OnDemandContainerReplicationSource(
ContainerSet containerSet) { ContainerController controller) {
this.containerSet = containerSet; this.controller = controller;
} }
@Override @Override
@ -57,7 +57,7 @@ public void prepare(long containerId) {
public void copyData(long containerId, OutputStream destination) public void copyData(long containerId, OutputStream destination)
throws IOException { throws IOException {
Container container = containerSet.getContainer(containerId); Container container = controller.getContainer(containerId);
Preconditions Preconditions
.checkNotNull(container, "Container is not found " + containerId); .checkNotNull(container, "Container is not found " + containerId);

View File

@ -17,7 +17,6 @@
package org.apache.hadoop.ozone.protocol.commands; package org.apache.hadoop.ozone.protocol.commands;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -30,14 +29,11 @@
public class CloseContainerCommand public class CloseContainerCommand
extends SCMCommand<CloseContainerCommandProto> { extends SCMCommand<CloseContainerCommandProto> {
private HddsProtos.ReplicationType replicationType; private final PipelineID pipelineID;
private PipelineID pipelineID;
public CloseContainerCommand(long containerID, public CloseContainerCommand(final long containerID,
HddsProtos.ReplicationType replicationType, final PipelineID pipelineID) {
PipelineID pipelineID) {
super(containerID); super(containerID);
this.replicationType = replicationType;
this.pipelineID = pipelineID; this.pipelineID = pipelineID;
} }
@ -65,7 +61,6 @@ public CloseContainerCommandProto getProto() {
return CloseContainerCommandProto.newBuilder() return CloseContainerCommandProto.newBuilder()
.setContainerID(getId()) .setContainerID(getId())
.setCmdId(getId()) .setCmdId(getId())
.setReplicationType(replicationType)
.setPipelineID(pipelineID.getProtobuf()) .setPipelineID(pipelineID.getProtobuf())
.build(); .build();
} }
@ -74,7 +69,6 @@ public static CloseContainerCommand getFromProtobuf(
CloseContainerCommandProto closeContainerProto) { CloseContainerCommandProto closeContainerProto) {
Preconditions.checkNotNull(closeContainerProto); Preconditions.checkNotNull(closeContainerProto);
return new CloseContainerCommand(closeContainerProto.getCmdId(), return new CloseContainerCommand(closeContainerProto.getCmdId(),
closeContainerProto.getReplicationType(),
PipelineID.getFromProtobuf(closeContainerProto.getPipelineID())); PipelineID.getFromProtobuf(closeContainerProto.getPipelineID()));
} }

View File

@ -140,8 +140,8 @@ message ContainerReplicaProto {
enum State { enum State {
OPEN = 1; OPEN = 1;
CLOSING = 2; CLOSING = 2;
CLOSED = 3; QUASI_CLOSED = 3;
QUASI_CLOSED = 4; CLOSED = 4;
UNHEALTHY = 5; UNHEALTHY = 5;
INVALID = 6; INVALID = 6;
} }
@ -289,9 +289,9 @@ This command asks the datanode to close a specific container.
*/ */
message CloseContainerCommandProto { message CloseContainerCommandProto {
required int64 containerID = 1; required int64 containerID = 1;
required hadoop.hdds.ReplicationType replicationType = 2; required PipelineID pipelineID = 2;
// cmdId will be removed
required int64 cmdId = 3; required int64 cmdId = 3;
required PipelineID pipelineID = 4;
} }
/** /**

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.container.common.impl; package org.apache.hadoop.ozone.container.common.impl;
import com.google.common.collect.Maps;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.conf.StorageUnit;
@ -25,6 +26,8 @@
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;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto
.ContainerProtos.ContainerType;
import org.apache.hadoop.hdds.protocol.datanode.proto import org.apache.hadoop.hdds.protocol.datanode.proto
.ContainerProtos.ContainerCommandResponseProto; .ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@ -33,7 +36,9 @@
.WriteChunkRequestProto; .WriteChunkRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerAction; .StorageContainerDatanodeProtocolProtos.ContainerAction;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.interfaces.Container; 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.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy; import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
@ -47,6 +52,7 @@
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Map;
import java.util.UUID; import java.util.UUID;
import static java.nio.charset.StandardCharsets.UTF_8; import static java.nio.charset.StandardCharsets.UTF_8;
@ -77,8 +83,15 @@ public void testContainerCloseActionWhenFull() throws IOException {
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(),
scmId.toString()); scmId.toString());
containerSet.addContainer(container); containerSet.addContainer(container);
ContainerMetrics metrics = ContainerMetrics.create(conf);
Map<ContainerType, Handler> handlers = Maps.newHashMap();
for (ContainerType containerType : ContainerType.values()) {
handlers.put(containerType,
Handler.getHandlerForContainerType(
containerType, conf, null, containerSet, volumeSet, metrics));
}
HddsDispatcher hddsDispatcher = new HddsDispatcher( HddsDispatcher hddsDispatcher = new HddsDispatcher(
conf, containerSet, volumeSet, context); conf, containerSet, volumeSet, handlers, context, metrics);
hddsDispatcher.setScmId(scmId.toString()); hddsDispatcher.setScmId(scmId.toString());
ContainerCommandResponseProto responseOne = hddsDispatcher.dispatch( ContainerCommandResponseProto responseOne = hddsDispatcher.dispatch(
getWriteChunkRequest(dd.getUuidString(), 1L, 1L)); getWriteChunkRequest(dd.getUuidString(), 1L, 1L));
@ -113,8 +126,15 @@ public void testCreateContainerWithWriteChunk() throws IOException {
ContainerSet containerSet = new ContainerSet(); ContainerSet containerSet = new ContainerSet();
VolumeSet volumeSet = new VolumeSet(dd.getUuidString(), conf); VolumeSet volumeSet = new VolumeSet(dd.getUuidString(), conf);
StateContext context = Mockito.mock(StateContext.class); StateContext context = Mockito.mock(StateContext.class);
HddsDispatcher hddsDispatcher = ContainerMetrics metrics = ContainerMetrics.create(conf);
new HddsDispatcher(conf, containerSet, volumeSet, context); Map<ContainerType, Handler> handlers = Maps.newHashMap();
for (ContainerType containerType : ContainerType.values()) {
handlers.put(containerType,
Handler.getHandlerForContainerType(
containerType, conf, null, containerSet, volumeSet, metrics));
}
HddsDispatcher hddsDispatcher = new HddsDispatcher(
conf, containerSet, volumeSet, handlers, context, metrics);
hddsDispatcher.setScmId(scmId.toString()); hddsDispatcher.setScmId(scmId.toString());
ContainerCommandRequestProto writeChunkRequest = ContainerCommandRequestProto writeChunkRequest =
getWriteChunkRequest(dd.getUuidString(), 1L, 1L); getWriteChunkRequest(dd.getUuidString(), 1L, 1L);

View File

@ -18,8 +18,10 @@
package org.apache.hadoop.ozone.container.common.interfaces; package org.apache.hadoop.ozone.container.common.interfaces;
import com.google.common.collect.Maps;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
@ -32,6 +34,8 @@
import org.junit.rules.Timeout; import org.junit.rules.Timeout;
import org.mockito.Mockito; import org.mockito.Mockito;
import java.util.Map;
/** /**
* Tests Handler interface. * Tests Handler interface.
*/ */
@ -50,8 +54,16 @@ public void setup() throws Exception {
this.conf = new Configuration(); this.conf = new Configuration();
this.containerSet = Mockito.mock(ContainerSet.class); this.containerSet = Mockito.mock(ContainerSet.class);
this.volumeSet = Mockito.mock(VolumeSet.class); this.volumeSet = Mockito.mock(VolumeSet.class);
ContainerMetrics metrics = ContainerMetrics.create(conf);
this.dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, null); Map<ContainerProtos.ContainerType, Handler> handlers = Maps.newHashMap();
for (ContainerProtos.ContainerType containerType :
ContainerProtos.ContainerType.values()) {
handlers.put(containerType,
Handler.getHandlerForContainerType(
containerType, conf, null, containerSet, volumeSet, metrics));
}
this.dispatcher = new HddsDispatcher(
conf, containerSet, volumeSet, handlers, null, metrics);
} }
@Test @Test

View File

@ -0,0 +1,210 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.commons.io.FileUtils;
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.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
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.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.junit.AfterClass;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.UUID;
/**
* Test cases to verify CloseContainerCommandHandler in datanode.
*/
public class TestCloseContainerCommandHandler {
private static final StateContext CONTEXT = Mockito.mock(StateContext.class);
private static File testDir;
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());
return new OzoneContainer(datanodeDetails, conf, CONTEXT);
}
@Test
public void testCloseContainerViaRatis()
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(1);
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 close the container on ratis.
final CloseContainerCommandHandler closeHandler =
new CloseContainerCommandHandler();
final CloseContainerCommand command = new CloseContainerCommand(
containerId.getId(), pipelineID);
final DatanodeStateMachine datanodeStateMachine = Mockito.mock(
DatanodeStateMachine.class);
Mockito.when(datanodeStateMachine.getDatanodeDetails())
.thenReturn(datanodeDetails);
Mockito.when(CONTEXT.getParent()).thenReturn(datanodeStateMachine);
closeHandler.handle(command, container, CONTEXT, null);
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
container.getContainerSet().getContainer(
containerId.getId()).getContainerState());
Mockito.verify(datanodeStateMachine, 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());
final DatanodeStateMachine datanodeStateMachine = Mockito.mock(
DatanodeStateMachine.class);
Mockito.when(datanodeStateMachine.getDatanodeDetails())
.thenReturn(datanodeDetails);
Mockito.when(CONTEXT.getParent()).thenReturn(datanodeStateMachine);
closeHandler.handle(command, container, CONTEXT, null);
Assert.assertEquals(ContainerProtos.ContainerDataProto.State.QUASI_CLOSED,
container.getContainerSet().getContainer(
containerId.getId()).getContainerState());
Mockito.verify(datanodeStateMachine, Mockito.times(2)).triggerHeartbeat();
container.stop();
}
/**
* Creates a random DatanodeDetails.
* @return DatanodeDetails
*/
private static DatanodeDetails randomDatanodeDetails() {
String ipAddress = "127.0.0.1";
DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
DatanodeDetails.Port.Name.STANDALONE, 0);
DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
DatanodeDetails.Port.Name.RATIS, 0);
DatanodeDetails.Port restPort = DatanodeDetails.newPort(
DatanodeDetails.Port.Name.REST, 0);
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
builder.setUuid(UUID.randomUUID().toString())
.setHostName("localhost")
.setIpAddress(ipAddress)
.addPort(containerPort)
.addPort(ratisPort)
.addPort(restPort);
return builder.build();
}
@AfterClass
public static void teardown() throws IOException {
FileUtils.deleteDirectory(testDir);
}
}

View File

@ -54,6 +54,7 @@
import java.io.File; import java.io.File;
import java.io.IOException;
import java.util.UUID; import java.util.UUID;
/** /**
@ -224,7 +225,7 @@ public void testVolumeSetInKeyValueHandler() throws Exception{
interval[0] = 2; interval[0] = 2;
ContainerMetrics metrics = new ContainerMetrics(interval); ContainerMetrics metrics = new ContainerMetrics(interval);
VolumeSet volumeSet = new VolumeSet(UUID.randomUUID().toString(), conf); VolumeSet volumeSet = new VolumeSet(UUID.randomUUID().toString(), conf);
KeyValueHandler keyValueHandler = new KeyValueHandler(conf, cset, KeyValueHandler keyValueHandler = new KeyValueHandler(conf, null, cset,
volumeSet, metrics); volumeSet, metrics);
assertEquals("org.apache.hadoop.ozone.container.common" + assertEquals("org.apache.hadoop.ozone.container.common" +
".volume.RoundRobinVolumeChoosingPolicy", ".volume.RoundRobinVolumeChoosingPolicy",
@ -235,7 +236,7 @@ public void testVolumeSetInKeyValueHandler() throws Exception{
conf.set(HDDS_DATANODE_VOLUME_CHOOSING_POLICY, conf.set(HDDS_DATANODE_VOLUME_CHOOSING_POLICY,
"org.apache.hadoop.ozone.container.common.impl.HddsDispatcher"); "org.apache.hadoop.ozone.container.common.impl.HddsDispatcher");
try { try {
new KeyValueHandler(conf, cset, volumeSet, metrics); new KeyValueHandler(conf, null, cset, volumeSet, metrics);
} catch (RuntimeException ex) { } catch (RuntimeException ex) {
GenericTestUtils.assertExceptionContains("class org.apache.hadoop" + GenericTestUtils.assertExceptionContains("class org.apache.hadoop" +
".ozone.container.common.impl.HddsDispatcher not org.apache" + ".ozone.container.common.impl.HddsDispatcher not org.apache" +
@ -261,7 +262,7 @@ private ContainerCommandRequestProto getDummyCommandRequestProto(
@Test @Test
public void testCloseInvalidContainer() { public void testCloseInvalidContainer() throws IOException {
long containerID = 1234L; long containerID = 1234L;
Configuration conf = new Configuration(); Configuration conf = new Configuration();
KeyValueContainerData kvData = new KeyValueContainerData(containerID, KeyValueContainerData kvData = new KeyValueContainerData(containerID,
@ -282,6 +283,7 @@ public void testCloseInvalidContainer() {
Mockito.when(handler.handleCloseContainer(any(), any())) Mockito.when(handler.handleCloseContainer(any(), any()))
.thenCallRealMethod(); .thenCallRealMethod();
doCallRealMethod().when(handler).closeContainer(any());
// Closing invalid container should return error response. // Closing invalid container should return error response.
ContainerProtos.ContainerCommandResponseProto response = ContainerProtos.ContainerCommandResponseProto response =
handler.handleCloseContainer(closeContainerRequest, container); handler.handleCloseContainer(closeContainerRequest, container);

View File

@ -172,7 +172,7 @@ private synchronized void preAllocateContainers(int count,
LOG.warn("Unable to allocate container."); LOG.warn("Unable to allocate container.");
} }
} catch (IOException ex) { } catch (IOException ex) {
LOG.warn("Unable to allocate container: {}", ex); LOG.warn("Unable to allocate container.", ex);
} }
} }
} }

View File

@ -74,8 +74,8 @@ public void onMessage(ContainerID containerID, EventPublisher publisher) {
if (container.getState() == LifeCycleState.CLOSING) { if (container.getState() == LifeCycleState.CLOSING) {
final CloseContainerCommand closeContainerCommand = final CloseContainerCommand closeContainerCommand =
new CloseContainerCommand(containerID.getId(), new CloseContainerCommand(
container.getReplicationType(), container.getPipelineID()); containerID.getId(), container.getPipelineID());
getNodes(container).forEach(node -> publisher.fireEvent( getNodes(container).forEach(node -> publisher.fireEvent(
DATANODE_COMMAND, DATANODE_COMMAND,

View File

@ -19,7 +19,6 @@
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
@ -960,7 +959,7 @@ public void testHandlingSCMCommandEvent() throws IOException {
TestUtils.getRandomPipelineReports()); TestUtils.getRandomPipelineReports());
eq.fireEvent(DATANODE_COMMAND, eq.fireEvent(DATANODE_COMMAND,
new CommandForDatanode<>(datanodeDetails.getUuid(), new CommandForDatanode<>(datanodeDetails.getUuid(),
new CloseContainerCommand(1L, ReplicationType.STAND_ALONE, new CloseContainerCommand(1L,
PipelineID.randomId()))); PipelineID.randomId())));
eq.processAll(1000L); eq.processAll(1000L);

View File

@ -21,7 +21,6 @@
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto. import org.apache.hadoop.hdds.protocol.proto.
StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto; StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
@ -66,6 +65,7 @@
import org.apache.hadoop.ozone.container.common.states.endpoint import org.apache.hadoop.ozone.container.common.states.endpoint
.VersionEndpointTask; .VersionEndpointTask;
import org.apache.hadoop.ozone.container.common.volume.HddsVolume; import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.protocol.commands.CommandStatus; import org.apache.hadoop.ozone.protocol.commands.CommandStatus;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
@ -75,6 +75,7 @@
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito;
import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS; import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
@ -309,8 +310,10 @@ private EndpointStateMachine registerTaskHelper(InetSocketAddress scmAddress,
OzoneContainer ozoneContainer = mock(OzoneContainer.class); OzoneContainer ozoneContainer = mock(OzoneContainer.class);
when(ozoneContainer.getNodeReport()).thenReturn(TestUtils when(ozoneContainer.getNodeReport()).thenReturn(TestUtils
.createNodeReport(getStorageReports(UUID.randomUUID()))); .createNodeReport(getStorageReports(UUID.randomUUID())));
when(ozoneContainer.getContainerReport()).thenReturn( ContainerController controller = Mockito.mock(ContainerController.class);
when(controller.getContainerReport()).thenReturn(
TestUtils.getRandomContainerReports(10)); TestUtils.getRandomContainerReports(10));
when(ozoneContainer.getController()).thenReturn(controller);
when(ozoneContainer.getPipelineReport()).thenReturn( when(ozoneContainer.getPipelineReport()).thenReturn(
TestUtils.getRandomPipelineReports()); TestUtils.getRandomPipelineReports());
RegisterEndpointTask endpointTask = RegisterEndpointTask endpointTask =
@ -433,7 +436,6 @@ private void addScmCommands() {
.setCloseContainerCommandProto( .setCloseContainerCommandProto(
CloseContainerCommandProto.newBuilder().setCmdId(1) CloseContainerCommandProto.newBuilder().setCmdId(1)
.setContainerID(1) .setContainerID(1)
.setReplicationType(ReplicationType.RATIS)
.setPipelineID(PipelineID.randomId().getProtobuf()) .setPipelineID(PipelineID.randomId().getProtobuf())
.build()) .build())
.setCommandType(Type.closeContainerCommand) .setCommandType(Type.closeContainerCommand)

View File

@ -42,6 +42,6 @@ Check webui static resources
Should contain ${result} 200 Should contain ${result} 200
Start freon testing Start freon testing
${result} = Execute ozone freon randomkeys --numOfVolumes 5 --numOfBuckets 5 --numOfKeys 5 --numOfThreads 10 ${result} = Execute ozone freon randomkeys --numOfVolumes 5 --numOfBuckets 5 --numOfKeys 5 --numOfThreads 1
Wait Until Keyword Succeeds 3min 10sec Should contain ${result} Number of Keys added: 125 Wait Until Keyword Succeeds 3min 10sec Should contain ${result} Number of Keys added: 125
Should Not Contain ${result} ERROR Should Not Contain ${result} ERROR

View File

@ -176,14 +176,14 @@ public void testContainerRandomPort() throws IOException {
TestUtils.randomDatanodeDetails(), ozoneConf) TestUtils.randomDatanodeDetails(), ozoneConf)
) { ) {
HashSet<Integer> ports = new HashSet<Integer>(); HashSet<Integer> ports = new HashSet<Integer>();
assertTrue(ports.add(sm1.getContainer().getContainerServerPort())); assertTrue(ports.add(sm1.getContainer().getReadChannel().getIPCPort()));
assertTrue(ports.add(sm2.getContainer().getContainerServerPort())); assertTrue(ports.add(sm2.getContainer().getReadChannel().getIPCPort()));
assertTrue(ports.add(sm3.getContainer().getContainerServerPort())); assertTrue(ports.add(sm3.getContainer().getReadChannel().getIPCPort()));
// Assert that ratis is also on a different port. // Assert that ratis is also on a different port.
assertTrue(ports.add(sm1.getContainer().getRatisContainerServerPort())); assertTrue(ports.add(sm1.getContainer().getWriteChannel().getIPCPort()));
assertTrue(ports.add(sm2.getContainer().getRatisContainerServerPort())); assertTrue(ports.add(sm2.getContainer().getWriteChannel().getIPCPort()));
assertTrue(ports.add(sm3.getContainer().getRatisContainerServerPort())); assertTrue(ports.add(sm3.getContainer().getWriteChannel().getIPCPort()));
} }
@ -199,9 +199,9 @@ public void testContainerRandomPort() throws IOException {
TestUtils.randomDatanodeDetails(), ozoneConf) TestUtils.randomDatanodeDetails(), ozoneConf)
) { ) {
HashSet<Integer> ports = new HashSet<Integer>(); HashSet<Integer> ports = new HashSet<Integer>();
assertTrue(ports.add(sm1.getContainer().getContainerServerPort())); assertTrue(ports.add(sm1.getContainer().getReadChannel().getIPCPort()));
assertFalse(ports.add(sm2.getContainer().getContainerServerPort())); assertFalse(ports.add(sm2.getContainer().getReadChannel().getIPCPort()));
assertFalse(ports.add(sm3.getContainer().getContainerServerPort())); assertFalse(ports.add(sm3.getContainer().getReadChannel().getIPCPort()));
assertEquals(ports.iterator().next().intValue(), assertEquals(ports.iterator().next().intValue(),
conf.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, conf.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT)); OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT));

View File

@ -134,7 +134,7 @@ public void testBlockWritesWithFlushAndClose() throws Exception {
.setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName) .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
.build(); .build();
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS); waitForContainerClose(keyName, key);
key.write(data); key.write(data);
key.flush(); key.flush();
key.close(); key.close();
@ -162,11 +162,12 @@ public void testBlockWritesCloseConsistency() throws Exception {
Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream); Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
//get the name of a valid container //get the name of a valid container
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName) OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
.setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS) .setBucketName(bucketName)
.setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName) .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
.build(); .build();
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS); waitForContainerClose(keyName, key);
key.close(); key.close();
// read the key from OM again and match the length.The length will still // read the key from OM again and match the length.The length will still
// be the equal to the original data size. // be the equal to the original data size.
@ -199,7 +200,7 @@ public void testMultiBlockWrites() throws Exception {
.setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName) .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
.build(); .build();
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS); waitForContainerClose(keyName, key);
// write 1 more block worth of data. It will fail and new block will be // write 1 more block worth of data. It will fail and new block will be
// allocated // allocated
key.write(ContainerTestHelper.getFixedLengthString(keyString, blockSize) key.write(ContainerTestHelper.getFixedLengthString(keyString, blockSize)
@ -249,7 +250,7 @@ public void testMultiBlockWrites2() throws Exception {
.setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName) .setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
.build(); .build();
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS); waitForContainerClose(keyName, key);
key.close(); key.close();
// read the key from OM again and match the length.The length will still // read the key from OM again and match the length.The length will still
@ -291,7 +292,7 @@ public void testMultiBlockWrites3() throws Exception {
.setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName) .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
.build(); .build();
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS); waitForContainerClose(keyName, key);
// write 3 more chunks worth of data. It will fail and new block will be // write 3 more chunks worth of data. It will fail and new block will be
// allocated. This write completes 4 blocks worth of data written to key // allocated. This write completes 4 blocks worth of data written to key
data = Arrays.copyOfRange(writtenData, 3 * blockSize + chunkSize, keyLen); data = Arrays.copyOfRange(writtenData, 3 * blockSize + chunkSize, keyLen);
@ -321,7 +322,7 @@ public void testMultiBlockWrites3() throws Exception {
} }
private void waitForContainerClose(String keyName, private void waitForContainerClose(String keyName,
OzoneOutputStream outputStream, HddsProtos.ReplicationType type) OzoneOutputStream outputStream)
throws Exception { throws Exception {
ChunkGroupOutputStream groupOutputStream = ChunkGroupOutputStream groupOutputStream =
(ChunkGroupOutputStream) outputStream.getOutputStream(); (ChunkGroupOutputStream) outputStream.getOutputStream();
@ -332,11 +333,10 @@ private void waitForContainerClose(String keyName,
containerIdList.add(info.getContainerID()); containerIdList.add(info.getContainerID());
} }
Assert.assertTrue(!containerIdList.isEmpty()); Assert.assertTrue(!containerIdList.isEmpty());
waitForContainerClose(type, containerIdList.toArray(new Long[0])); waitForContainerClose(containerIdList.toArray(new Long[0]));
} }
private void waitForContainerClose(HddsProtos.ReplicationType type, private void waitForContainerClose(Long... containerIdList)
Long... containerIdList)
throws ContainerNotFoundException, PipelineNotFoundException, throws ContainerNotFoundException, PipelineNotFoundException,
TimeoutException, InterruptedException { TimeoutException, InterruptedException {
List<Pipeline> pipelineList = new ArrayList<>(); List<Pipeline> pipelineList = new ArrayList<>();
@ -358,7 +358,7 @@ private void waitForContainerClose(HddsProtos.ReplicationType type,
// send the order to close the container // send the order to close the container
cluster.getStorageContainerManager().getScmNodeManager() cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(details.getUuid(), .addDatanodeCommand(details.getUuid(),
new CloseContainerCommand(containerID, type, pipeline.getId())); new CloseContainerCommand(containerID, pipeline.getId()));
} }
} }
int index = 0; int index = 0;
@ -413,7 +413,7 @@ public void testDiscardPreallocatedBlocks() throws Exception {
.getPipeline(container.getPipelineID()); .getPipeline(container.getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(1, datanodes.size()); Assert.assertEquals(1, datanodes.size());
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS); waitForContainerClose(keyName, key);
dataString = dataString =
ContainerTestHelper.getFixedLengthString(keyString, (1 * blockSize)); ContainerTestHelper.getFixedLengthString(keyString, (1 * blockSize));
data = dataString.getBytes(UTF_8); data = dataString.getBytes(UTF_8);
@ -459,7 +459,7 @@ public void testBlockWriteViaRatis() throws Exception {
.build(); .build();
Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream); Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS); waitForContainerClose(keyName, key);
// Again Write the Data. This will throw an exception which will be handled // Again Write the Data. This will throw an exception which will be handled
// and new blocks will be allocated // and new blocks will be allocated
key.write(data); key.write(data);

View File

@ -22,9 +22,11 @@
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;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
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;
@ -32,7 +34,6 @@
import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.container.common.impl.ContainerData; import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
@ -63,8 +64,9 @@ public class TestCloseContainerByPipeline {
@BeforeClass @BeforeClass
public static void init() throws Exception { public static void init() throws Exception {
conf = new OzoneConfiguration(); conf = new OzoneConfiguration();
conf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, "1");
cluster = MiniOzoneCluster.newBuilder(conf) cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(3).build(); .setNumDatanodes(9).build();
cluster.waitForClusterToBeReady(); cluster.waitForClusterToBeReady();
//the easiest way to create an open container is creating a key //the easiest way to create an open container is creating a key
client = OzoneClientFactory.getClient(conf); client = OzoneClientFactory.getClient(conf);
@ -86,7 +88,7 @@ public static void shutdown() {
@Test @Test
public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception { public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test") OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
.createKey("testCloseContainer", 1024, ReplicationType.STAND_ALONE, .createKey("standalone", 1024, ReplicationType.RATIS,
ReplicationFactor.ONE); ReplicationFactor.ONE);
key.write("standalone".getBytes()); key.write("standalone".getBytes());
key.close(); key.close();
@ -94,10 +96,9 @@ public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
//get the name of a valid container //get the name of a valid container
OmKeyArgs keyArgs = OmKeyArgs keyArgs =
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test") new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
.setType(HddsProtos.ReplicationType.STAND_ALONE) .setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024) .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
.setKeyName("testCloseContainer").build(); .setKeyName("standalone").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);
@ -127,8 +128,7 @@ public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
//send the order to close the container //send the order to close the container
cluster.getStorageContainerManager().getScmNodeManager() cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(datanodeDetails.getUuid(), .addDatanodeCommand(datanodeDetails.getUuid(),
new CloseContainerCommand(containerID, new CloseContainerCommand(containerID, pipeline.getId()));
HddsProtos.ReplicationType.STAND_ALONE, pipeline.getId()));
GenericTestUtils GenericTestUtils
.waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails), .waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails),
500, 5 * 1000); 500, 5 * 1000);
@ -142,7 +142,7 @@ public void testCloseContainerViaStandAlone()
throws IOException, TimeoutException, InterruptedException { throws IOException, TimeoutException, InterruptedException {
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test") OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
.createKey("standalone", 1024, ReplicationType.STAND_ALONE, .createKey("standalone", 1024, ReplicationType.RATIS,
ReplicationFactor.ONE); ReplicationFactor.ONE);
key.write("standalone".getBytes()); key.write("standalone".getBytes());
key.close(); key.close();
@ -150,7 +150,7 @@ public void testCloseContainerViaStandAlone()
//get the name of a valid container //get the name of a valid container
OmKeyArgs keyArgs = OmKeyArgs keyArgs =
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test") new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
.setType(HddsProtos.ReplicationType.STAND_ALONE) .setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024) .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
.setKeyName("standalone").build(); .setKeyName("standalone").build();
@ -170,30 +170,20 @@ public void testCloseContainerViaStandAlone()
Assert Assert
.assertFalse(isContainerClosed(cluster, containerID, datanodeDetails)); .assertFalse(isContainerClosed(cluster, containerID, datanodeDetails));
GenericTestUtils.LogCapturer logCapturer = // Send the order to close the container, give random pipeline id so that
GenericTestUtils.LogCapturer.captureLogs(OzoneContainer.LOG); // the container will not be closed via RATIS
//send the order to close the container
cluster.getStorageContainerManager().getScmNodeManager() cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(datanodeDetails.getUuid(), .addDatanodeCommand(datanodeDetails.getUuid(),
new CloseContainerCommand(containerID, new CloseContainerCommand(containerID, PipelineID.randomId()));
HddsProtos.ReplicationType.STAND_ALONE, pipeline.getId()));
// The log will appear after the state changed to closed in standalone,
// wait for the log to ensure the operation has been done.
GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains(
"submitting CloseContainer request over STAND_ALONE server for"
+ " container " + containerID), 500, 5 * 1000);
//double check if it's really closed (waitFor also throws an exception) //double check if it's really closed (waitFor also throws an exception)
Assert.assertTrue(isContainerClosed(cluster, containerID, datanodeDetails)); // TODO: change the below line after implementing QUASI_CLOSED to CLOSED
Assert.assertTrue(logCapturer.getOutput().contains( // logic. The container will be QUASI closed as of now
"submitting CloseContainer request over STAND_ALONE server for" GenericTestUtils
+ " container " + containerID)); .waitFor(() -> isContainerQuasiClosed(
// Make sure it was really closed via StandAlone not Ratis server cluster, containerID, datanodeDetails), 500, 5 * 1000);
Assert.assertFalse((logCapturer.getOutput().contains( Assert.assertTrue(
"submitting CloseContainer request over RATIS server for container " isContainerQuasiClosed(cluster, containerID, datanodeDetails));
+ containerID)));
logCapturer.stopCapturing();
} }
@Test @Test
@ -224,18 +214,14 @@ public void testCloseContainerViaRatis() throws IOException,
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(3, datanodes.size()); Assert.assertEquals(3, datanodes.size());
GenericTestUtils.LogCapturer logCapturer =
GenericTestUtils.LogCapturer.captureLogs(OzoneContainer.LOG);
for (DatanodeDetails details : datanodes) { for (DatanodeDetails details : datanodes) {
Assert.assertFalse(isContainerClosed(cluster, containerID, details)); Assert.assertFalse(isContainerClosed(cluster, containerID, details));
//send the order to close the container //send the order to close the container
cluster.getStorageContainerManager().getScmNodeManager() cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(details.getUuid(), .addDatanodeCommand(details.getUuid(),
new CloseContainerCommand(containerID, new CloseContainerCommand(containerID, pipeline.getId()));
HddsProtos.ReplicationType.RATIS, pipeline.getId()));
} }
// Make sure that it is CLOSED
for (DatanodeDetails datanodeDetails : datanodes) { for (DatanodeDetails datanodeDetails : datanodes) {
GenericTestUtils.waitFor( GenericTestUtils.waitFor(
() -> isContainerClosed(cluster, containerID, datanodeDetails), 500, () -> isContainerClosed(cluster, containerID, datanodeDetails), 500,
@ -244,14 +230,6 @@ public void testCloseContainerViaRatis() throws IOException,
Assert.assertTrue(isContainerClosed(cluster, Assert.assertTrue(isContainerClosed(cluster,
containerID, datanodeDetails)); containerID, datanodeDetails));
} }
// Make sure it was really closed via Ratis not STAND_ALONE server
Assert.assertFalse(logCapturer.getOutput().contains(
"submitting CloseContainer request over STAND_ALONE "
+ "server for container " + containerID));
Assert.assertTrue((logCapturer.getOutput().contains(
"submitting CloseContainer request over RATIS server for container "
+ containerID)));
logCapturer.stopCapturing();
} }
private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID, private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
@ -267,4 +245,18 @@ private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
} }
return false; return false;
} }
private Boolean isContainerQuasiClosed(MiniOzoneCluster miniCluster,
long containerID, DatanodeDetails datanode) {
ContainerData containerData;
for (HddsDatanodeService datanodeService : miniCluster.getHddsDatanodes()) {
if (datanode.equals(datanodeService.getDatanodeDetails())) {
containerData =
datanodeService.getDatanodeStateMachine().getContainer()
.getContainerSet().getContainer(containerID).getContainerData();
return containerData.isQuasiClosed();
}
}
return false;
}
} }

View File

@ -95,8 +95,7 @@ public void test() throws IOException, TimeoutException, InterruptedException,
//send the order to close the container //send the order to close the container
cluster.getStorageContainerManager().getScmNodeManager() cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(datanodeDetails.getUuid(), .addDatanodeCommand(datanodeDetails.getUuid(),
new CloseContainerCommand(containerId.getId(), new CloseContainerCommand(containerId.getId(), pipeline.getId()));
HddsProtos.ReplicationType.STAND_ALONE, pipeline.getId()));
GenericTestUtils.waitFor(() -> GenericTestUtils.waitFor(() ->
isContainerClosed(cluster, containerId.getId()), isContainerClosed(cluster, containerId.getId()),

View File

@ -21,6 +21,7 @@
import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges; import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import com.google.common.collect.Maps;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@ -36,12 +37,15 @@
import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.ContainerTestHelper; import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.ozone.container.replication.GrpcReplicationService; import org.apache.hadoop.ozone.container.replication.GrpcReplicationService;
import org.apache.hadoop.ozone.container.replication.OnDemandContainerReplicationSource; import org.apache.hadoop.ozone.container.replication.OnDemandContainerReplicationSource;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
@ -49,6 +53,7 @@
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.util.Map;
import java.util.UUID; import java.util.UUID;
/** /**
@ -57,9 +62,9 @@
public class TestContainerMetrics { public class TestContainerMetrics {
private GrpcReplicationService createReplicationService( private GrpcReplicationService createReplicationService(
ContainerSet containerSet) { ContainerController controller) {
return new GrpcReplicationService( return new GrpcReplicationService(
new OnDemandContainerReplicationSource(containerSet)); new OnDemandContainerReplicationSource(controller));
} }
@Test @Test
@ -85,12 +90,21 @@ public void testContainerMetrics() throws Exception {
VolumeSet volumeSet = new VolumeSet( VolumeSet volumeSet = new VolumeSet(
datanodeDetails.getUuidString(), conf); datanodeDetails.getUuidString(), conf);
ContainerSet containerSet = new ContainerSet(); ContainerSet containerSet = new ContainerSet();
ContainerMetrics metrics = ContainerMetrics.create(conf);
Map<ContainerProtos.ContainerType, Handler> handlers = Maps.newHashMap();
for (ContainerProtos.ContainerType containerType :
ContainerProtos.ContainerType.values()) {
handlers.put(containerType,
Handler.getHandlerForContainerType(
containerType, conf, null, containerSet, volumeSet, metrics));
}
HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet, HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet,
volumeSet, null); volumeSet, handlers, null, metrics);
dispatcher.setScmId(UUID.randomUUID().toString()); dispatcher.setScmId(UUID.randomUUID().toString());
server = new XceiverServerGrpc(datanodeDetails, conf, dispatcher, server = new XceiverServerGrpc(datanodeDetails, conf, dispatcher,
createReplicationService(containerSet)); createReplicationService(new ContainerController(
containerSet, handlers)));
client = new XceiverClientGrpc(pipeline, conf); client = new XceiverClientGrpc(pipeline, conf);
server.start(); server.start();

View File

@ -18,11 +18,14 @@
package org.apache.hadoop.ozone.container.server; package org.apache.hadoop.ozone.container.server;
import com.google.common.collect.Maps;
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.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
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.volume.VolumeSet; import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
import org.apache.hadoop.ozone.container.replication.GrpcReplicationService; import org.apache.hadoop.ozone.container.replication.GrpcReplicationService;
import org.apache.hadoop.ozone.container.replication.OnDemandContainerReplicationSource; import org.apache.hadoop.ozone.container.replication.OnDemandContainerReplicationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@ -57,6 +60,7 @@
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
import static org.apache.ratis.rpc.SupportedRpcType.GRPC; import static org.apache.ratis.rpc.SupportedRpcType.GRPC;
import static org.apache.ratis.rpc.SupportedRpcType.NETTY; import static org.apache.ratis.rpc.SupportedRpcType.NETTY;
@ -71,15 +75,17 @@ public class TestContainerServer {
= GenericTestUtils.getTestDir("dfs").getAbsolutePath() + File.separator; = GenericTestUtils.getTestDir("dfs").getAbsolutePath() + File.separator;
private GrpcReplicationService createReplicationService( private GrpcReplicationService createReplicationService(
ContainerSet containerSet) { ContainerController containerController) {
return new GrpcReplicationService( return new GrpcReplicationService(
new OnDemandContainerReplicationSource(containerSet)); new OnDemandContainerReplicationSource(containerController));
} }
@Test @Test
public void testClientServer() throws Exception { public void testClientServer() throws Exception {
DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails(); DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
ContainerSet containerSet = new ContainerSet(); ContainerSet containerSet = new ContainerSet();
ContainerController controller = new ContainerController(
containerSet, null);
runTestClientServer(1, (pipeline, conf) -> conf runTestClientServer(1, (pipeline, conf) -> conf
.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, .setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getFirstNode() pipeline.getFirstNode()
@ -87,7 +93,7 @@ public void testClientServer() throws Exception {
XceiverClientGrpc::new, XceiverClientGrpc::new,
(dn, conf) -> new XceiverServerGrpc(datanodeDetails, conf, (dn, conf) -> new XceiverServerGrpc(datanodeDetails, conf,
new TestContainerDispatcher(), new TestContainerDispatcher(),
createReplicationService(containerSet)), (dn, p) -> { createReplicationService(controller)), (dn, p) -> {
}); });
} }
@ -185,12 +191,22 @@ public void testClientServerWithContainerDispatcher() throws Exception {
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue()); .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
ContainerSet containerSet = new ContainerSet(); ContainerSet containerSet = new ContainerSet();
VolumeSet volumeSet = mock(VolumeSet.class);
ContainerMetrics metrics = ContainerMetrics.create(conf);
Map<ContainerProtos.ContainerType, Handler> handlers = Maps.newHashMap();
for (ContainerProtos.ContainerType containerType :
ContainerProtos.ContainerType.values()) {
handlers.put(containerType,
Handler.getHandlerForContainerType(
containerType, conf, null, containerSet, volumeSet, metrics));
}
HddsDispatcher dispatcher = new HddsDispatcher( HddsDispatcher dispatcher = new HddsDispatcher(
conf, mock(ContainerSet.class), mock(VolumeSet.class), null); conf, containerSet, volumeSet, handlers, null, metrics);
dispatcher.init(); dispatcher.init();
DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails(); DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
server = new XceiverServerGrpc(datanodeDetails, conf, dispatcher, server = new XceiverServerGrpc(datanodeDetails, conf, dispatcher,
createReplicationService(containerSet)); createReplicationService(
new ContainerController(containerSet, null)));
client = new XceiverClientGrpc(pipeline, conf); client = new XceiverClientGrpc(pipeline, conf);
server.start(); server.start();

View File

@ -17,9 +17,12 @@
*/ */
package org.apache.hadoop.ozone.genesis; package org.apache.hadoop.ozone.genesis;
import com.google.common.collect.Maps;
import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.statemachine import org.apache.hadoop.ozone.container.common.statemachine
.DatanodeStateMachine.DatanodeStates; .DatanodeStateMachine.DatanodeStates;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@ -44,6 +47,7 @@
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Random; import java.util.Random;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -100,9 +104,18 @@ public void initialize() throws IOException {
ContainerSet containerSet = new ContainerSet(); ContainerSet containerSet = new ContainerSet();
VolumeSet volumeSet = new VolumeSet(datanodeUuid, conf); VolumeSet volumeSet = new VolumeSet(datanodeUuid, conf);
StateContext context = new StateContext(
dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, conf, DatanodeStates.RUNNING, null);
new StateContext(conf, DatanodeStates.RUNNING, null)); ContainerMetrics metrics = ContainerMetrics.create(conf);
Map<ContainerProtos.ContainerType, Handler> handlers = Maps.newHashMap();
for (ContainerProtos.ContainerType containerType :
ContainerProtos.ContainerType.values()) {
handlers.put(containerType,
Handler.getHandlerForContainerType(
containerType, conf, context, containerSet, volumeSet, metrics));
}
dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, handlers,
context, metrics);
dispatcher.init(); dispatcher.init();
containerCount = new AtomicInteger(); containerCount = new AtomicInteger();

View File

@ -21,7 +21,6 @@
import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.container.common.transport import org.apache.hadoop.ozone.container.common.transport
.server.XceiverServerSpi; .server.XceiverServerSpi;
@ -128,7 +127,7 @@ private void startFreon() throws Exception {
private StateMachine getStateMachine() throws Exception { private StateMachine getStateMachine() throws Exception {
XceiverServerSpi server = XceiverServerSpi server =
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine(). cluster.getHddsDatanodes().get(0).getDatanodeStateMachine().
getContainer().getServer(HddsProtos.ReplicationType.RATIS); getContainer().getWriteChannel();
RaftServerProxy proxy = RaftServerProxy proxy =
(RaftServerProxy)(((XceiverServerRatis)server).getServer()); (RaftServerProxy)(((XceiverServerRatis)server).getServer());
RaftGroupId groupId = proxy.getGroupIds().iterator().next(); RaftGroupId groupId = proxy.getGroupIds().iterator().next();