HDDS-576. Move ContainerWithPipeline creation to RPC endpoint.

Contributed by Nanda kumar.
This commit is contained in:
Nanda kumar 2018-11-12 23:32:31 +05:30
parent 42f3a7082a
commit 18fe65d756
33 changed files with 380 additions and 390 deletions

View File

@ -102,6 +102,6 @@ public final class ContainerID implements Comparable<ContainerID> {
@Override @Override
public String toString() { public String toString() {
return "id=" + id; return "#" + id;
} }
} }

View File

@ -111,24 +111,18 @@ message NodePool {
*/ */
enum LifeCycleState { enum LifeCycleState {
ALLOCATED = 1; OPEN = 1;
CREATING = 2; // Used for container allocated/created by different client. CLOSING = 2;
OPEN =3; // Mostly an update to SCM via HB or client call. CLOSED = 3;
CLOSING = 4; DELETING = 4;
CLOSED = 5; // !!State after this has not been used yet. DELETED = 5; // object is deleted.
DELETING = 6;
DELETED = 7; // object is deleted.
} }
enum LifeCycleEvent { enum LifeCycleEvent {
CREATE = 1; // A request to client to create this object FINALIZE = 1;
CREATED = 2; CLOSE = 2; // !!Event after this has not been used yet.
FINALIZE = 3; DELETE = 3;
CLOSE = 4; // !!Event after this has not been used yet. CLEANUP = 4;
UPDATE = 5;
TIMEOUT = 6; // creation has timed out from SCM's View.
DELETE = 7;
CLEANUP = 8;
} }
message ContainerInfoProto { message ContainerInfoProto {

View File

@ -84,7 +84,7 @@ public class CloseContainerCommandHandler implements CommandHandler {
cmdExecuted = false; cmdExecuted = false;
return; return;
} }
if (container.getContainerData().isClosed()) { if (!container.getContainerData().isClosed()) {
LOG.debug("Closing container {}.", containerID); LOG.debug("Closing container {}.", containerID);
HddsProtos.PipelineID pipelineID = closeContainerProto.getPipelineID(); HddsProtos.PipelineID pipelineID = closeContainerProto.getPipelineID();
HddsProtos.ReplicationType replicationType = HddsProtos.ReplicationType replicationType =

View File

@ -32,6 +32,7 @@ public class CommandForDatanode<T extends GeneratedMessage> implements
private final SCMCommand<T> command; private final SCMCommand<T> command;
// TODO: Command for datanode should take DatanodeDetails as parameter.
public CommandForDatanode(UUID datanodeId, SCMCommand<T> command) { public CommandForDatanode(UUID datanodeId, SCMCommand<T> command) {
this.datanodeId = datanodeId; this.datanodeId = datanodeId;
this.command = command; this.command = command;

View File

@ -26,13 +26,15 @@ import org.apache.hadoop.hdds.scm.ScmUtils;
import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck; import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock; import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.metrics2.util.MBeans;
@ -70,6 +72,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
// Currently only user of the block service is Ozone, CBlock manages blocks // Currently only user of the block service is Ozone, CBlock manages blocks
// by itself and does not rely on the Block service offered by SCM. // by itself and does not rely on the Block service offered by SCM.
private final PipelineManager pipelineManager;
private final ContainerManager containerManager; private final ContainerManager containerManager;
private final long containerSize; private final long containerSize;
@ -87,14 +90,16 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
* *
* @param conf - configuration. * @param conf - configuration.
* @param nodeManager - node manager. * @param nodeManager - node manager.
* @param pipelineManager - pipeline manager.
* @param containerManager - container manager. * @param containerManager - container manager.
* @param eventPublisher - event publisher. * @param eventPublisher - event publisher.
* @throws IOException * @throws IOException
*/ */
public BlockManagerImpl(final Configuration conf, public BlockManagerImpl(final Configuration conf,
final NodeManager nodeManager, final ContainerManager containerManager, final NodeManager nodeManager, final PipelineManager pipelineManager,
EventPublisher eventPublisher) final ContainerManager containerManager, EventPublisher eventPublisher)
throws IOException { throws IOException {
this.pipelineManager = pipelineManager;
this.containerManager = containerManager; this.containerManager = containerManager;
this.containerSize = (long)conf.getStorageSize( this.containerSize = (long)conf.getStorageSize(
@ -155,16 +160,15 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
* @throws IOException * @throws IOException
*/ */
private synchronized void preAllocateContainers(int count, private synchronized void preAllocateContainers(int count,
ReplicationType type, ReplicationFactor factor, String owner) ReplicationType type, ReplicationFactor factor, String owner) {
throws IOException {
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
ContainerWithPipeline containerWithPipeline; ContainerInfo containerInfo;
try { try {
// TODO: Fix this later when Ratis is made the Default. // TODO: Fix this later when Ratis is made the Default.
containerWithPipeline = containerManager.allocateContainer( containerInfo = containerManager.allocateContainer(
type, factor, owner); type, factor, owner);
if (containerWithPipeline == null) { if (containerInfo == null) {
LOG.warn("Unable to allocate container."); LOG.warn("Unable to allocate container.");
} }
} catch (IOException ex) { } catch (IOException ex) {
@ -206,11 +210,11 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
use different kind of policies. use different kind of policies.
*/ */
ContainerWithPipeline containerWithPipeline; ContainerInfo containerInfo;
// look for OPEN containers that match the criteria. // look for OPEN containers that match the criteria.
containerWithPipeline = containerManager containerInfo = containerManager
.getMatchingContainerWithPipeline(size, owner, type, factor, .getMatchingContainer(size, owner, type, factor,
HddsProtos.LifeCycleState.OPEN); HddsProtos.LifeCycleState.OPEN);
// We did not find OPEN Containers. This generally means // We did not find OPEN Containers. This generally means
@ -221,27 +225,27 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
// Even though we have already checked the containers in OPEN // Even though we have already checked the containers in OPEN
// state, we have to check again as we only hold a read lock. // state, we have to check again as we only hold a read lock.
// Some other thread might have pre-allocated container in meantime. // Some other thread might have pre-allocated container in meantime.
if (containerWithPipeline == null) { if (containerInfo == null) {
synchronized (this) { synchronized (this) {
if (!containerManager.getContainers(HddsProtos.LifeCycleState.OPEN) if (!containerManager.getContainers(HddsProtos.LifeCycleState.OPEN)
.isEmpty()) { .isEmpty()) {
containerWithPipeline = containerManager containerInfo = containerManager
.getMatchingContainerWithPipeline(size, owner, type, factor, .getMatchingContainer(size, owner, type, factor,
HddsProtos.LifeCycleState.OPEN); HddsProtos.LifeCycleState.OPEN);
} }
if (containerWithPipeline == null) { if (containerInfo == null) {
preAllocateContainers(containerProvisionBatchSize, type, factor, preAllocateContainers(containerProvisionBatchSize, type, factor,
owner); owner);
containerWithPipeline = containerManager containerInfo = containerManager
.getMatchingContainerWithPipeline(size, owner, type, factor, .getMatchingContainer(size, owner, type, factor,
HddsProtos.LifeCycleState.OPEN); HddsProtos.LifeCycleState.OPEN);
} }
} }
} }
if (containerWithPipeline != null) { if (containerInfo != null) {
return newBlock(containerWithPipeline, HddsProtos.LifeCycleState.OPEN); return newBlock(containerInfo);
} }
// we have tried all strategies we know and but somehow we are not able // we have tried all strategies we know and but somehow we are not able
@ -255,29 +259,26 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
/** /**
* newBlock - returns a new block assigned to a container. * newBlock - returns a new block assigned to a container.
* *
* @param containerWithPipeline - Container Info. * @param containerInfo - Container Info.
* @param state - Current state of the container.
* @return AllocatedBlock * @return AllocatedBlock
*/ */
private AllocatedBlock newBlock(ContainerWithPipeline containerWithPipeline, private AllocatedBlock newBlock(ContainerInfo containerInfo) {
HddsProtos.LifeCycleState state) throws IOException { try {
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo(); final Pipeline pipeline = pipelineManager
if (containerWithPipeline.getPipeline().getNodes().size() == 0) { .getPipeline(containerInfo.getPipelineID());
LOG.error("Pipeline Machine count is zero."); // TODO : Revisit this local ID allocation when HA is added.
long localID = UniqueId.next();
long containerID = containerInfo.getContainerID();
AllocatedBlock.Builder abb = new AllocatedBlock.Builder()
.setContainerBlockID(new ContainerBlockID(containerID, localID))
.setPipeline(pipeline);
LOG.trace("New block allocated : {} Container ID: {}", localID,
containerID);
return abb.build();
} catch (PipelineNotFoundException ex) {
LOG.error("Pipeline Machine count is zero.", ex);
return null; return null;
} }
// TODO : Revisit this local ID allocation when HA is added.
long localID = UniqueId.next();
long containerID = containerInfo.getContainerID();
AllocatedBlock.Builder abb =
new AllocatedBlock.Builder()
.setContainerBlockID(new ContainerBlockID(containerID, localID))
.setPipeline(containerWithPipeline.getPipeline());
LOG.trace("New block allocated : {} Container ID: {}", localID,
containerID);
return abb.build();
} }
/** /**

View File

@ -18,8 +18,8 @@ package org.apache.hadoop.hdds.scm.block;
import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ArrayListMultimap;
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.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
@ -30,8 +30,7 @@ import java.util.Set;
import java.util.UUID; import java.util.UUID;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
/** /**
* A wrapper class to hold info about datanode and all deleted block * A wrapper class to hold info about datanode and all deleted block
@ -58,31 +57,28 @@ public class DatanodeDeletedBlockTransactions {
public boolean addTransaction(DeletedBlocksTransaction tx, public boolean addTransaction(DeletedBlocksTransaction tx,
Set<UUID> dnsWithTransactionCommitted) { Set<UUID> dnsWithTransactionCommitted) {
Pipeline pipeline = null;
try { try {
ContainerWithPipeline containerWithPipeline = boolean success = false;
containerManager.getContainerWithPipeline( final ContainerID id = ContainerID.valueof(tx.getContainerID());
ContainerID.valueof(tx.getContainerID())); final ContainerInfo container = containerManager.getContainer(id);
if (containerWithPipeline.getContainerInfo().isOpen() final Set<ContainerReplica> replicas = containerManager
|| containerWithPipeline.getPipeline().isEmpty()) { .getContainerReplicas(id);
return false; if (!container.isOpen()) {
for (ContainerReplica replica : replicas) {
UUID dnID = replica.getDatanodeDetails().getUuid();
if (dnsWithTransactionCommitted == null ||
!dnsWithTransactionCommitted.contains(dnID)) {
// Transaction need not be sent to dns which have
// already committed it
success = addTransactionToDN(dnID, tx);
}
}
} }
pipeline = containerWithPipeline.getPipeline(); return success;
} catch (IOException e) { } catch (IOException e) {
SCMBlockDeletingService.LOG.warn("Got container info error.", e); SCMBlockDeletingService.LOG.warn("Got container info error.", e);
return false; return false;
} }
boolean success = false;
for (DatanodeDetails dd : pipeline.getNodes()) {
UUID dnID = dd.getUuid();
if (dnsWithTransactionCommitted == null ||
!dnsWithTransactionCommitted.contains(dnID)) {
// Transaction need not be sent to dns which have already committed it
success = addTransactionToDN(dnID, tx);
}
}
return success;
} }
private boolean addTransactionToDN(UUID dnID, DeletedBlocksTransaction tx) { private boolean addTransactionToDN(UUID dnID, DeletedBlocksTransaction tx) {

View File

@ -30,8 +30,9 @@ import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.scm.command import org.apache.hadoop.hdds.scm.command
.CommandStatusReportHandler.DeleteBlockStatus; .CommandStatusReportHandler.DeleteBlockStatus;
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.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.server.ServerUtils; import org.apache.hadoop.hdds.server.ServerUtils;
import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.server.events.EventPublisher;
@ -49,7 +50,6 @@ import org.slf4j.LoggerFactory;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -101,7 +101,7 @@ public class DeletedBlockLogImpl
private Map<Long, Set<UUID>> transactionToDNsCommitMap; private Map<Long, Set<UUID>> transactionToDNsCommitMap;
public DeletedBlockLogImpl(Configuration conf, public DeletedBlockLogImpl(Configuration conf,
ContainerManager containerManager) throws IOException { ContainerManager containerManager) throws IOException {
maxRetry = conf.getInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY, maxRetry = conf.getInt(OZONE_SCM_BLOCK_DELETION_MAX_RETRY,
OZONE_SCM_BLOCK_DELETION_MAX_RETRY_DEFAULT); OZONE_SCM_BLOCK_DELETION_MAX_RETRY_DEFAULT);
@ -249,7 +249,8 @@ public class DeletedBlockLogImpl
long txID = transactionResult.getTxID(); long txID = transactionResult.getTxID();
// set of dns which have successfully committed transaction txId. // set of dns which have successfully committed transaction txId.
dnsWithCommittedTxn = transactionToDNsCommitMap.get(txID); dnsWithCommittedTxn = transactionToDNsCommitMap.get(txID);
Long containerId = transactionResult.getContainerID(); final ContainerID containerId = ContainerID.valueof(
transactionResult.getContainerID());
if (dnsWithCommittedTxn == null) { if (dnsWithCommittedTxn == null) {
LOG.warn("Transaction txId={} commit by dnId={} for containerID={} " LOG.warn("Transaction txId={} commit by dnId={} for containerID={} "
+ "failed. Corresponding entry not found.", txID, dnID, + "failed. Corresponding entry not found.", txID, dnID,
@ -258,16 +259,17 @@ public class DeletedBlockLogImpl
} }
dnsWithCommittedTxn.add(dnID); dnsWithCommittedTxn.add(dnID);
Pipeline pipeline = final ContainerInfo container = containerManager
containerManager.getContainerWithPipeline( .getContainer(containerId);
ContainerID.valueof(containerId)).getPipeline(); final Set<ContainerReplica> replicas =
Collection<DatanodeDetails> containerDnsDetails = pipeline.getNodes(); containerManager.getContainerReplicas(containerId);
// The delete entry can be safely removed from the log if all the // The delete entry can be safely removed from the log if all the
// corresponding nodes commit the txn. It is required to check that // corresponding nodes commit the txn. It is required to check that
// the nodes returned in the pipeline match the replication factor. // the nodes returned in the pipeline match the replication factor.
if (min(containerDnsDetails.size(), dnsWithCommittedTxn.size()) if (min(replicas.size(), dnsWithCommittedTxn.size())
>= pipeline.getFactor().getNumber()) { >= container.getReplicationFactor().getNumber()) {
List<UUID> containerDns = containerDnsDetails.stream() List<UUID> containerDns = replicas.stream()
.map(ContainerReplica::getDatanodeDetails)
.map(DatanodeDetails::getUuid) .map(DatanodeDetails::getUuid)
.collect(Collectors.toList()); .collect(Collectors.toList());
if (dnsWithCommittedTxn.containsAll(containerDns)) { if (dnsWithCommittedTxn.containsAll(containerDns)) {

View File

@ -17,9 +17,14 @@
package org.apache.hadoop.hdds.scm.container; package org.apache.hadoop.hdds.scm.container;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import java.util.List;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import java.util.stream.Collectors;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
@ -42,78 +47,67 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
public static final Logger LOG = public static final Logger LOG =
LoggerFactory.getLogger(CloseContainerEventHandler.class); LoggerFactory.getLogger(CloseContainerEventHandler.class);
private final PipelineManager pipelineManager;
private final ContainerManager containerManager; private final ContainerManager containerManager;
public CloseContainerEventHandler(ContainerManager containerManager) { public CloseContainerEventHandler(final PipelineManager pipelineManager,
final ContainerManager containerManager) {
this.pipelineManager = pipelineManager;
this.containerManager = containerManager; this.containerManager = containerManager;
} }
@Override @Override
public void onMessage(ContainerID containerID, EventPublisher publisher) { public void onMessage(ContainerID containerID, EventPublisher publisher) {
LOG.info("Close container Event triggered for container : {}", containerID);
LOG.info("Close container Event triggered for container : {}",
containerID.getId());
ContainerWithPipeline containerWithPipeline;
ContainerInfo info;
try { try {
containerWithPipeline = // If the container is in OPEN state, FINALIZE it.
containerManager.getContainerWithPipeline(containerID); if (containerManager.getContainer(containerID).getState()
info = containerWithPipeline.getContainerInfo(); == LifeCycleState.OPEN) {
if (info == null) { containerManager.updateContainerState(
LOG.error("Failed to update the container state. Container with id : {}" containerID, LifeCycleEvent.FINALIZE);
+ " does not exist", containerID.getId());
return;
} }
} catch (IOException e) {
LOG.error("Failed to update the container state. Container with id : {} "
+ "does not exist", containerID.getId(), e);
return;
}
HddsProtos.LifeCycleState state = info.getState(); // ContainerInfo has to read again after the above state change.
try { final ContainerInfo container = containerManager
switch (state) { .getContainer(containerID);
case OPEN: // Send close command to datanodes, if the container is in CLOSING state
containerManager.updateContainerState(containerID, if (container.getState() == LifeCycleState.CLOSING) {
HddsProtos.LifeCycleEvent.FINALIZE);
fireCloseContainerEvents(containerWithPipeline, info, publisher); final CloseContainerCommand closeContainerCommand =
break; new CloseContainerCommand(containerID.getId(),
case CLOSING: container.getReplicationType(), container.getPipelineID());
fireCloseContainerEvents(containerWithPipeline, info, publisher);
break; getNodes(container).forEach(node -> publisher.fireEvent(
case CLOSED: DATANODE_COMMAND,
case DELETING: new CommandForDatanode<>(node.getUuid(), closeContainerCommand)));
case DELETED: } else {
LOG.info("Cannot close container #{}, it is already in {} state.", LOG.warn("Cannot close container {}, which is in {} state.",
containerID.getId(), state); containerID, container.getState());
break;
default:
throw new IOException("Invalid container state for container #"
+ containerID);
} }
} catch (IOException ex) { } catch (IOException ex) {
LOG.error("Failed to update the container state for container #{}" LOG.error("Failed to close the container {}.", containerID, ex);
+ containerID, ex);
} }
} }
private void fireCloseContainerEvents( /**
ContainerWithPipeline containerWithPipeline, ContainerInfo info, * Returns the list of Datanodes where this container lives.
EventPublisher publisher) { *
ContainerID containerID = info.containerID(); * @param container ContainerInfo
// fire events. * @return list of DatanodeDetails
CloseContainerCommand closeContainerCommand = * @throws ContainerNotFoundException
new CloseContainerCommand(containerID.getId(), */
info.getReplicationType(), info.getPipelineID()); private List<DatanodeDetails> getNodes(final ContainerInfo container)
throws ContainerNotFoundException {
Pipeline pipeline = containerWithPipeline.getPipeline(); try {
pipeline.getNodes().stream() return pipelineManager.getPipeline(container.getPipelineID()).getNodes();
.map(node -> } catch (PipelineNotFoundException ex) {
new CommandForDatanode<>(node.getUuid(), closeContainerCommand)) // Use container replica if the pipeline is not available.
.forEach(command -> publisher.fireEvent(DATANODE_COMMAND, command)); return containerManager.getContainerReplicas(container.containerID())
.stream()
LOG.trace("Issuing {} on Pipeline {} for container", closeContainerCommand, .map(ContainerReplica::getDatanodeDetails)
pipeline, containerID); .collect(Collectors.toList());
}
} }
} }

View File

@ -19,9 +19,7 @@ package org.apache.hadoop.hdds.scm.container;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
@ -62,16 +60,6 @@ public interface ContainerManager extends Closeable {
ContainerInfo getContainer(ContainerID containerID) ContainerInfo getContainer(ContainerID containerID)
throws ContainerNotFoundException; throws ContainerNotFoundException;
/**
* Returns the ContainerInfo from the container ID.
*
* @param containerID - ID of container.
* @return - ContainerWithPipeline such as creation state and the pipeline.
* @throws IOException
*/
ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
throws ContainerNotFoundException, PipelineNotFoundException;
/** /**
* Returns containers under certain conditions. * Returns containers under certain conditions.
* Search container IDs from start ID(exclusive), * Search container IDs from start ID(exclusive),
@ -94,10 +82,10 @@ public interface ContainerManager extends Closeable {
* *
* @param replicationFactor - replication factor of the container. * @param replicationFactor - replication factor of the container.
* @param owner * @param owner
* @return - ContainerWithPipeline. * @return - ContainerInfo.
* @throws IOException * @throws IOException
*/ */
ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type, ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
HddsProtos.ReplicationFactor replicationFactor, String owner) HddsProtos.ReplicationFactor replicationFactor, String owner)
throws IOException; throws IOException;
@ -158,10 +146,10 @@ public interface ContainerManager extends Closeable {
throws IOException; throws IOException;
/** /**
* Returns the ContainerWithPipeline. * Returns the ContainerInfo.
* @return NodeManager * @return NodeManager
*/ */
ContainerWithPipeline getMatchingContainerWithPipeline(long size, ContainerInfo getMatchingContainer(long size,
String owner, ReplicationType type, ReplicationFactor factor, String owner, ReplicationType type, ReplicationFactor factor,
LifeCycleState state) throws IOException; LifeCycleState state) throws IOException;
} }

View File

@ -227,7 +227,9 @@ public class ContainerStateManager {
final List<Pipeline> pipelines = pipelineManager final List<Pipeline> pipelines = pipelineManager
.getPipelines(type, replicationFactor, Pipeline.PipelineState.OPEN); .getPipelines(type, replicationFactor, Pipeline.PipelineState.OPEN);
if (pipelines.isEmpty()) { if (pipelines.isEmpty()) {
throw new IOException("Could not allocate container"); throw new IOException("Could not allocate container. Cannot get any" +
" matching pipeline for Type:" + type +
", Factor:" + replicationFactor + ", State:PipelineState.OPEN");
} }
pipeline = pipelines.get((int) containerCount.get() % pipelines.size()); pipeline = pipelines.get((int) containerCount.get() % pipelines.size());
} }

View File

@ -21,15 +21,11 @@ import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@ -184,42 +180,6 @@ public class SCMContainerManager implements ContainerManager {
return containerStateManager.getContainer(containerID); return containerStateManager.getContainer(containerID);
} }
/**
* Returns the ContainerInfo and pipeline from the containerID. If container
* has no available replicas in datanodes it returns pipeline with no
* datanodes and empty leaderID . Pipeline#isEmpty can be used to check for
* an empty pipeline.
*
* @param containerID - ID of container.
* @return - ContainerWithPipeline such as creation state and the pipeline.
* @throws IOException
*/
@Override
public ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
throws ContainerNotFoundException, PipelineNotFoundException {
lock.lock();
try {
final ContainerInfo contInfo = getContainer(containerID);
Pipeline pipeline;
if (contInfo.isOpen()) {
// If pipeline with given pipeline Id already exist return it
pipeline = pipelineManager.getPipeline(contInfo.getPipelineID());
} else {
// For close containers create pipeline from datanodes with replicas
Set<ContainerReplica> dnWithReplicas = containerStateManager
.getContainerReplicas(contInfo.containerID());
List<DatanodeDetails> dns =
dnWithReplicas.stream().map(ContainerReplica::getDatanodeDetails)
.collect(Collectors.toList());
pipeline = pipelineManager.createPipeline(ReplicationType.STAND_ALONE,
contInfo.getReplicationFactor(), dns);
}
return new ContainerWithPipeline(contInfo, pipeline);
} finally {
lock.unlock();
}
}
/** /**
* {@inheritDoc} * {@inheritDoc}
*/ */
@ -261,16 +221,13 @@ public class SCMContainerManager implements ContainerManager {
* @throws IOException - Exception * @throws IOException - Exception
*/ */
@Override @Override
public ContainerWithPipeline allocateContainer(final ReplicationType type, public ContainerInfo allocateContainer(final ReplicationType type,
final ReplicationFactor replicationFactor, final String owner) final ReplicationFactor replicationFactor, final String owner)
throws IOException { throws IOException {
lock.lock(); lock.lock();
try { try {
final ContainerInfo containerInfo; containerInfo = containerStateManager final ContainerInfo containerInfo; containerInfo = containerStateManager
.allocateContainer(pipelineManager, type, replicationFactor, owner); .allocateContainer(pipelineManager, type, replicationFactor, owner);
final Pipeline pipeline = pipelineManager.getPipeline(
containerInfo.getPipelineID());
try { try {
final byte[] containerIDBytes = Longs.toByteArray( final byte[] containerIDBytes = Longs.toByteArray(
containerInfo.getContainerID()); containerInfo.getContainerID());
@ -286,7 +243,7 @@ public class SCMContainerManager implements ContainerManager {
} }
throw ex; throw ex;
} }
return new ContainerWithPipeline(containerInfo, pipeline); return containerInfo;
} finally { } finally {
lock.unlock(); lock.unlock();
} }
@ -366,12 +323,8 @@ public class SCMContainerManager implements ContainerManager {
break; break;
case CLOSE: case CLOSE:
break; break;
case UPDATE:
break;
case DELETE: case DELETE:
break; break;
case TIMEOUT:
break;
case CLEANUP: case CLEANUP:
break; break;
default: default:
@ -434,17 +387,11 @@ public class SCMContainerManager implements ContainerManager {
* @param state - State of the Container-- {Open, Allocated etc.} * @param state - State of the Container-- {Open, Allocated etc.}
* @return ContainerInfo, null if there is no match found. * @return ContainerInfo, null if there is no match found.
*/ */
public ContainerWithPipeline getMatchingContainerWithPipeline( public ContainerInfo getMatchingContainer(
final long sizeRequired, String owner, ReplicationType type, final long sizeRequired, String owner, ReplicationType type,
ReplicationFactor factor, LifeCycleState state) throws IOException { ReplicationFactor factor, LifeCycleState state) throws IOException {
ContainerInfo containerInfo = containerStateManager return containerStateManager.getMatchingContainer(
.getMatchingContainer(sizeRequired, owner, type, factor, state); sizeRequired, owner, type, factor, state);
if (containerInfo == null) {
return null;
}
Pipeline pipeline = pipelineManager
.getPipeline(containerInfo.getPipelineID());
return new ContainerWithPipeline(containerInfo, pipeline);
} }
/** /**

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hdds.scm.ScmUtils;
import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck; import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
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;
@ -61,6 +62,7 @@ import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.stream.Collectors;
import static org.apache.hadoop.hdds.protocol.proto import static org.apache.hadoop.hdds.protocol.proto
.StorageContainerLocationProtocolProtos .StorageContainerLocationProtocolProtos
@ -161,11 +163,13 @@ public class SCMClientProtocolServer implements
replicationType, HddsProtos.ReplicationFactor factor, replicationType, HddsProtos.ReplicationFactor factor,
String owner) throws IOException { String owner) throws IOException {
ScmUtils.preCheck(ScmOps.allocateContainer, chillModePrecheck); ScmUtils.preCheck(ScmOps.allocateContainer, chillModePrecheck);
String remoteUser = getRpcRemoteUsername(); getScm().checkAdminAccess(getRpcRemoteUsername());
getScm().checkAdminAccess(remoteUser);
return scm.getContainerManager() final ContainerInfo container = scm.getContainerManager()
.allocateContainer(replicationType, factor, owner); .allocateContainer(replicationType, factor, owner);
final Pipeline pipeline = scm.getPipelineManager()
.getPipeline(container.getPipelineID());
return new ContainerWithPipeline(container, pipeline);
} }
@Override @Override
@ -191,8 +195,26 @@ public class SCMClientProtocolServer implements
} }
} }
getScm().checkAdminAccess(null); getScm().checkAdminAccess(null);
return scm.getContainerManager()
.getContainerWithPipeline(ContainerID.valueof(containerID)); final ContainerID id = ContainerID.valueof(containerID);
final ContainerInfo container = scm.getContainerManager().getContainer(id);
final Pipeline pipeline;
if (container.isOpen()) {
// Ratis pipeline
pipeline = scm.getPipelineManager()
.getPipeline(container.getPipelineID());
} else {
pipeline = scm.getPipelineManager().createPipeline(
HddsProtos.ReplicationType.STAND_ALONE,
container.getReplicationFactor(),
scm.getContainerManager()
.getContainerReplicas(id).stream()
.map(ContainerReplica::getDatanodeDetails)
.collect(Collectors.toList()));
}
return new ContainerWithPipeline(container, pipeline);
} }
/** /**

View File

@ -207,13 +207,13 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
pipelineManager = new SCMPipelineManager(conf, scmNodeManager, eventQueue); pipelineManager = new SCMPipelineManager(conf, scmNodeManager, eventQueue);
containerManager = new SCMContainerManager( containerManager = new SCMContainerManager(
conf, scmNodeManager, pipelineManager, eventQueue); conf, scmNodeManager, pipelineManager, eventQueue);
scmBlockManager = new BlockManagerImpl( scmBlockManager = new BlockManagerImpl(conf, scmNodeManager,
conf, scmNodeManager, containerManager, eventQueue); pipelineManager, containerManager, eventQueue);
replicationStatus = new ReplicationActivityStatus(); replicationStatus = new ReplicationActivityStatus();
CloseContainerEventHandler closeContainerHandler = CloseContainerEventHandler closeContainerHandler =
new CloseContainerEventHandler(containerManager); new CloseContainerEventHandler(pipelineManager, containerManager);
NodeReportHandler nodeReportHandler = NodeReportHandler nodeReportHandler =
new NodeReportHandler(scmNodeManager); new NodeReportHandler(scmNodeManager);
PipelineReportHandler pipelineReportHandler = PipelineReportHandler pipelineReportHandler =

View File

@ -413,7 +413,7 @@ public final class TestUtils {
throws IOException { throws IOException {
return containerManager return containerManager
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE, .allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE, "root").getContainerInfo(); HddsProtos.ReplicationFactor.THREE, "root");
} }

View File

@ -91,7 +91,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
mapping = new SCMContainerManager(conf, nodeManager, pipelineManager, mapping = new SCMContainerManager(conf, nodeManager, pipelineManager,
eventQueue); eventQueue);
blockManager = new BlockManagerImpl(conf, blockManager = new BlockManagerImpl(conf,
nodeManager, mapping, eventQueue); nodeManager, pipelineManager, mapping, eventQueue);
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager); eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);
eventQueue.addHandler(SCMEvents.START_REPLICATION, this); eventQueue.addHandler(SCMEvents.START_REPLICATION, this);
if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,

View File

@ -20,11 +20,14 @@ package org.apache.hadoop.hdds.scm.block;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.RandomUtils; import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager; import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
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.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
@ -51,11 +54,13 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Random; import java.util.Random;
import java.util.Set;
import java.util.UUID; import java.util.UUID;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -99,21 +104,23 @@ public class TestDeletedBlockLog {
DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString()) DatanodeDetails.newBuilder().setUuid(UUID.randomUUID().toString())
.build()); .build());
ContainerInfo containerInfo = final ContainerInfo container =
new ContainerInfo.Builder().setContainerID(1).build(); new ContainerInfo.Builder().setContainerID(1)
Pipeline pipeline = Pipeline.newBuilder() .setReplicationFactor(ReplicationFactor.THREE)
.setType(ReplicationType.RATIS) .setState(HddsProtos.LifeCycleState.CLOSED)
.setFactor(ReplicationFactor.THREE) .build();
.setState(Pipeline.PipelineState.CLOSED) final Set<ContainerReplica> replicaSet = dnList.stream()
.setId(PipelineID.randomId()) .map(datanodeDetails -> ContainerReplica.newBuilder()
.setNodes(dnList) .setContainerID(container.containerID())
.build(); .setContainerState(ContainerReplicaProto.State.OPEN)
ContainerWithPipeline containerWithPipeline = .setDatanodeDetails(datanodeDetails)
new ContainerWithPipeline(containerInfo, pipeline); .build())
when(containerManager.getContainerWithPipeline(anyObject())) .collect(Collectors.toSet());
.thenReturn(containerWithPipeline);
when(containerManager.getContainerReplicas(anyObject()))
.thenReturn(replicaSet);
when(containerManager.getContainer(anyObject())) when(containerManager.getContainer(anyObject()))
.thenReturn(containerInfo); .thenReturn(container);
} }
@After @After
@ -383,8 +390,7 @@ public class TestDeletedBlockLog {
private void mockContainerInfo(long containerID, DatanodeDetails dd) private void mockContainerInfo(long containerID, DatanodeDetails dd)
throws IOException { throws IOException {
List<DatanodeDetails> dns = new ArrayList<>(); List<DatanodeDetails> dns = Collections.singletonList(dd);
dns.add(dd);
Pipeline pipeline = Pipeline.newBuilder() Pipeline pipeline = Pipeline.newBuilder()
.setType(ReplicationType.STAND_ALONE) .setType(ReplicationType.STAND_ALONE)
.setFactor(ReplicationFactor.ONE) .setFactor(ReplicationFactor.ONE)
@ -399,11 +405,18 @@ public class TestDeletedBlockLog {
.setReplicationFactor(pipeline.getFactor()); .setReplicationFactor(pipeline.getFactor());
ContainerInfo containerInfo = builder.build(); ContainerInfo containerInfo = builder.build();
ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline(
containerInfo, pipeline);
Mockito.doReturn(containerInfo).when(containerManager) Mockito.doReturn(containerInfo).when(containerManager)
.getContainer(ContainerID.valueof(containerID)); .getContainer(ContainerID.valueof(containerID));
Mockito.doReturn(containerWithPipeline).when(containerManager)
.getContainerWithPipeline(ContainerID.valueof(containerID)); final Set<ContainerReplica> replicaSet = dns.stream()
.map(datanodeDetails -> ContainerReplica.newBuilder()
.setContainerID(containerInfo.containerID())
.setContainerState(ContainerReplicaProto.State.OPEN)
.setDatanodeDetails(datanodeDetails)
.build())
.collect(Collectors.toSet());
when(containerManager.getContainerReplicas(
ContainerID.valueof(containerID)))
.thenReturn(replicaSet);
} }
} }

View File

@ -24,8 +24,6 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
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.container.common.helpers
.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.hdds.server.events.EventQueue;
@ -51,6 +49,7 @@ public class TestCloseContainerEventHandler {
private static Configuration configuration; private static Configuration configuration;
private static MockNodeManager nodeManager; private static MockNodeManager nodeManager;
private static PipelineManager pipelineManager;
private static SCMContainerManager containerManager; private static SCMContainerManager containerManager;
private static long size; private static long size;
private static File testDir; private static File testDir;
@ -66,14 +65,14 @@ public class TestCloseContainerEventHandler {
configuration configuration
.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath()); .set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
nodeManager = new MockNodeManager(true, 10); nodeManager = new MockNodeManager(true, 10);
PipelineManager pipelineManager = pipelineManager =
new SCMPipelineManager(configuration, nodeManager, eventQueue); new SCMPipelineManager(configuration, nodeManager, eventQueue);
containerManager = new containerManager = new
SCMContainerManager(configuration, nodeManager, SCMContainerManager(configuration, nodeManager,
pipelineManager, new EventQueue()); pipelineManager, new EventQueue());
eventQueue = new EventQueue(); eventQueue = new EventQueue();
eventQueue.addHandler(CLOSE_CONTAINER, eventQueue.addHandler(CLOSE_CONTAINER,
new CloseContainerEventHandler(containerManager)); new CloseContainerEventHandler(pipelineManager, containerManager));
eventQueue.addHandler(DATANODE_COMMAND, nodeManager); eventQueue.addHandler(DATANODE_COMMAND, nodeManager);
} }
@ -105,19 +104,18 @@ public class TestCloseContainerEventHandler {
new ContainerID(id)); new ContainerID(id));
eventQueue.processAll(1000); eventQueue.processAll(1000);
Assert.assertTrue(logCapturer.getOutput() Assert.assertTrue(logCapturer.getOutput()
.contains("Failed to update the container state")); .contains("Failed to close the container"));
} }
@Test @Test
public void testCloseContainerEventWithValidContainers() throws IOException { public void testCloseContainerEventWithValidContainers() throws IOException {
ContainerWithPipeline containerWithPipeline = containerManager ContainerInfo container = containerManager
.allocateContainer(HddsProtos.ReplicationType.RATIS, .allocateContainer(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.ONE, "ozone"); HddsProtos.ReplicationFactor.ONE, "ozone");
ContainerID id = new ContainerID( ContainerID id = container.containerID();
containerWithPipeline.getContainerInfo().getContainerID()); DatanodeDetails datanode = pipelineManager
DatanodeDetails datanode = .getPipeline(container.getPipelineID()).getFirstNode();
containerWithPipeline.getPipeline().getFirstNode();
int closeCount = nodeManager.getCommandCount(datanode); int closeCount = nodeManager.getCommandCount(datanode);
eventQueue.fireEvent(CLOSE_CONTAINER, id); eventQueue.fireEvent(CLOSE_CONTAINER, id);
eventQueue.processAll(1000); eventQueue.processAll(1000);
@ -132,23 +130,22 @@ public class TestCloseContainerEventHandler {
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
.captureLogs(CloseContainerEventHandler.LOG); .captureLogs(CloseContainerEventHandler.LOG);
ContainerWithPipeline containerWithPipeline = containerManager ContainerInfo container = containerManager
.allocateContainer(HddsProtos.ReplicationType.RATIS, .allocateContainer(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.THREE, "ozone"); HddsProtos.ReplicationFactor.THREE, "ozone");
ContainerID id = new ContainerID( ContainerID id = container.containerID();
containerWithPipeline.getContainerInfo().getContainerID());
int[] closeCount = new int[3]; int[] closeCount = new int[3];
eventQueue.fireEvent(CLOSE_CONTAINER, id); eventQueue.fireEvent(CLOSE_CONTAINER, id);
eventQueue.processAll(1000); eventQueue.processAll(1000);
int i = 0; int i = 0;
for (DatanodeDetails details : containerWithPipeline.getPipeline() for (DatanodeDetails details : pipelineManager
.getNodes()) { .getPipeline(container.getPipelineID()).getNodes()) {
closeCount[i] = nodeManager.getCommandCount(details); closeCount[i] = nodeManager.getCommandCount(details);
i++; i++;
} }
i = 0; i = 0;
for (DatanodeDetails details : containerWithPipeline.getPipeline() for (DatanodeDetails details : pipelineManager
.getNodes()) { .getPipeline(container.getPipelineID()).getNodes()) {
Assert.assertEquals(closeCount[i], nodeManager.getCommandCount(details)); Assert.assertEquals(closeCount[i], nodeManager.getCommandCount(details));
i++; i++;
} }
@ -156,8 +153,8 @@ public class TestCloseContainerEventHandler {
eventQueue.processAll(1000); eventQueue.processAll(1000);
i = 0; i = 0;
// Make sure close is queued for each datanode on the pipeline // Make sure close is queued for each datanode on the pipeline
for (DatanodeDetails details : containerWithPipeline.getPipeline() for (DatanodeDetails details : pipelineManager
.getNodes()) { .getPipeline(container.getPipelineID()).getNodes()) {
Assert.assertEquals(closeCount[i] + 1, Assert.assertEquals(closeCount[i] + 1,
nodeManager.getCommandCount(details)); nodeManager.getCommandCount(details));
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING, Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,

View File

@ -100,14 +100,14 @@ public class TestContainerReportHandler implements EventPublisher {
ContainerInfo cont1 = containerManager ContainerInfo cont1 = containerManager
.allocateContainer(ReplicationType.STAND_ALONE, .allocateContainer(ReplicationType.STAND_ALONE,
ReplicationFactor.THREE, "root").getContainerInfo(); ReplicationFactor.THREE, "root");
ContainerInfo cont2 = containerManager ContainerInfo cont2 = containerManager
.allocateContainer(ReplicationType.STAND_ALONE, .allocateContainer(ReplicationType.STAND_ALONE,
ReplicationFactor.THREE, "root").getContainerInfo(); ReplicationFactor.THREE, "root");
// Open Container // Open Container
ContainerInfo cont3 = containerManager ContainerInfo cont3 = containerManager
.allocateContainer(ReplicationType.STAND_ALONE, .allocateContainer(ReplicationType.STAND_ALONE,
ReplicationFactor.THREE, "root").getContainerInfo(); ReplicationFactor.THREE, "root");
long c1 = cont1.getContainerID(); long c1 = cont1.getContainerID();
long c2 = cont2.getContainerID(); long c2 = cont2.getContainerID();

View File

@ -23,7 +23,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
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;
@ -44,11 +43,13 @@ import org.junit.rules.ExpectedException;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator;
import java.util.Random; import java.util.Random;
import java.util.Set; import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
/** /**
* Tests for Container ContainerManager. * Tests for Container ContainerManager.
@ -109,7 +110,7 @@ public class TestSCMContainerManager {
@Test @Test
public void testallocateContainer() throws Exception { public void testallocateContainer() throws Exception {
ContainerWithPipeline containerInfo = containerManager.allocateContainer( ContainerInfo containerInfo = containerManager.allocateContainer(
xceiverClientManager.getType(), xceiverClientManager.getType(),
xceiverClientManager.getFactor(), xceiverClientManager.getFactor(),
containerOwner); containerOwner);
@ -126,14 +127,15 @@ public class TestSCMContainerManager {
*/ */
Set<UUID> pipelineList = new TreeSet<>(); Set<UUID> pipelineList = new TreeSet<>();
for (int x = 0; x < 30; x++) { for (int x = 0; x < 30; x++) {
ContainerWithPipeline containerInfo = containerManager.allocateContainer( ContainerInfo containerInfo = containerManager.allocateContainer(
xceiverClientManager.getType(), xceiverClientManager.getType(),
xceiverClientManager.getFactor(), xceiverClientManager.getFactor(),
containerOwner); containerOwner);
Assert.assertNotNull(containerInfo); Assert.assertNotNull(containerInfo);
Assert.assertNotNull(containerInfo.getPipeline()); Assert.assertNotNull(containerInfo.getPipelineID());
pipelineList.add(containerInfo.getPipeline().getFirstNode() pipelineList.add(pipelineManager.getPipeline(
containerInfo.getPipelineID()).getFirstNode()
.getUuid()); .getUuid());
} }
Assert.assertTrue(pipelineList.size() > 5); Assert.assertTrue(pipelineList.size() > 5);
@ -141,32 +143,27 @@ public class TestSCMContainerManager {
@Test @Test
public void testGetContainer() throws IOException { public void testGetContainer() throws IOException {
ContainerWithPipeline containerInfo = containerManager.allocateContainer( ContainerInfo containerInfo = containerManager.allocateContainer(
xceiverClientManager.getType(), xceiverClientManager.getType(),
xceiverClientManager.getFactor(), xceiverClientManager.getFactor(),
containerOwner); containerOwner);
Pipeline pipeline = containerInfo.getPipeline(); Assert.assertNotNull(containerInfo);
Pipeline pipeline = pipelineManager
.getPipeline(containerInfo.getPipelineID());
Assert.assertNotNull(pipeline); Assert.assertNotNull(pipeline);
Pipeline newPipeline = containerInfo.getPipeline(); Assert.assertEquals(containerInfo,
Assert.assertEquals(pipeline.getFirstNode().getUuid(), containerManager.getContainer(containerInfo.containerID()));
newPipeline.getFirstNode().getUuid());
} }
@Test @Test
public void testGetContainerWithPipeline() throws Exception { public void testGetContainerWithPipeline() throws Exception {
ContainerWithPipeline containerWithPipeline = containerManager ContainerInfo contInfo = containerManager
.allocateContainer(xceiverClientManager.getType(), .allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner); xceiverClientManager.getFactor(), containerOwner);
ContainerInfo contInfo = containerWithPipeline.getContainerInfo();
// Add dummy replicas for container. // Add dummy replicas for container.
DatanodeDetails dn1 = DatanodeDetails.newBuilder() Iterator<DatanodeDetails> nodes = pipelineManager
.setHostName("host1") .getPipeline(contInfo.getPipelineID()).getNodes().iterator();
.setIpAddress("1.1.1.1") DatanodeDetails dn1 = nodes.next();
.setUuid(UUID.randomUUID().toString()).build();
DatanodeDetails dn2 = DatanodeDetails.newBuilder()
.setHostName("host2")
.setIpAddress("2.2.2.2")
.setUuid(UUID.randomUUID().toString()).build();
containerManager.updateContainerState(contInfo.containerID(), containerManager.updateContainerState(contInfo.containerID(),
LifeCycleEvent.FINALIZE); LifeCycleEvent.FINALIZE);
containerManager containerManager
@ -180,27 +177,21 @@ public class TestSCMContainerManager {
ContainerReplica.newBuilder().setContainerID(contInfo.containerID()) ContainerReplica.newBuilder().setContainerID(contInfo.containerID())
.setContainerState(ContainerReplicaProto.State.CLOSED) .setContainerState(ContainerReplicaProto.State.CLOSED)
.setDatanodeDetails(dn1).build()); .setDatanodeDetails(dn1).build());
containerManager.updateContainerReplica(contInfo.containerID(),
ContainerReplica.newBuilder().setContainerID(contInfo.containerID())
.setContainerState(ContainerReplicaProto.State.CLOSED)
.setDatanodeDetails(dn2).build());
Assert.assertEquals(2, Assert.assertEquals(1,
containerManager.getContainerReplicas( containerManager.getContainerReplicas(
finalContInfo.containerID()).size()); finalContInfo.containerID()).size());
contInfo = containerManager.getContainer(contInfo.containerID()); contInfo = containerManager.getContainer(contInfo.containerID());
Assert.assertEquals(contInfo.getState(), LifeCycleState.CLOSED); Assert.assertEquals(contInfo.getState(), LifeCycleState.CLOSED);
Pipeline pipeline = containerWithPipeline.getPipeline(); // After closing the container, we should get the replica and construct
pipelineManager.finalizePipeline(pipeline.getId()); // standalone pipeline. No more ratis pipeline.
ContainerWithPipeline containerWithPipeline2 = containerManager Set<DatanodeDetails> replicaNodes = containerManager
.getContainerWithPipeline(contInfo.containerID()); .getContainerReplicas(contInfo.containerID())
pipeline = containerWithPipeline2.getPipeline(); .stream().map(ContainerReplica::getDatanodeDetails)
Assert.assertNotEquals(containerWithPipeline, containerWithPipeline2); .collect(Collectors.toSet());
Assert.assertNotNull("Pipeline should not be null", pipeline); Assert.assertTrue(replicaNodes.contains(dn1));
Assert.assertTrue(pipeline.getNodes().contains(dn1));
Assert.assertTrue(pipeline.getNodes().contains(dn2));
} }
@Test @Test
@ -232,11 +223,9 @@ public class TestSCMContainerManager {
private ContainerInfo createContainer() private ContainerInfo createContainer()
throws IOException { throws IOException {
nodeManager.setChillmode(false); nodeManager.setChillmode(false);
ContainerWithPipeline containerWithPipeline = containerManager return containerManager
.allocateContainer(xceiverClientManager.getType(), .allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner); xceiverClientManager.getFactor(), containerOwner);
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
return containerInfo;
} }
} }

View File

@ -25,8 +25,8 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
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.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager; import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.placement.algorithms import org.apache.hadoop.hdds.scm.container.placement.algorithms
.ContainerPlacementPolicy; .ContainerPlacementPolicy;
import org.apache.hadoop.hdds.scm.container.placement.algorithms import org.apache.hadoop.hdds.scm.container.placement.algorithms
@ -155,12 +155,13 @@ public class TestContainerPlacement {
assertEquals(remaining * nodeCount, assertEquals(remaining * nodeCount,
(long) nodeManager.getStats().getRemaining().get()); (long) nodeManager.getStats().getRemaining().get());
ContainerWithPipeline containerWithPipeline = containerManager ContainerInfo container = containerManager
.allocateContainer( .allocateContainer(
xceiverClientManager.getType(), xceiverClientManager.getType(),
xceiverClientManager.getFactor(), "OZONE"); xceiverClientManager.getFactor(), "OZONE");
assertEquals(xceiverClientManager.getFactor().getNumber(), assertEquals(xceiverClientManager.getFactor().getNumber(),
containerWithPipeline.getPipeline().getNodes().size()); containerManager.getContainerReplicas(
container.containerID()).size());
} finally { } finally {
IOUtils.closeQuietly(containerManager); IOUtils.closeQuietly(containerManager);
IOUtils.closeQuietly(nodeManager); IOUtils.closeQuietly(nodeManager);

View File

@ -21,6 +21,7 @@ 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.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.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline; .ContainerWithPipeline;
@ -61,8 +62,11 @@ public class TestNode2PipelineMap {
cluster.waitForClusterToBeReady(); cluster.waitForClusterToBeReady();
scm = cluster.getStorageContainerManager(); scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager(); containerManager = scm.getContainerManager();
ratisContainer = containerManager.allocateContainer( pipelineManager = scm.getPipelineManager();
ContainerInfo containerInfo = containerManager.allocateContainer(
RATIS, THREE, "testOwner"); RATIS, THREE, "testOwner");
ratisContainer = new ContainerWithPipeline(containerInfo,
pipelineManager.getPipeline(containerInfo.getPipelineID()));
pipelineManager = scm.getPipelineManager(); pipelineManager = scm.getPipelineManager();
} }

View File

@ -22,14 +22,13 @@ 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.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
@ -48,8 +47,8 @@ public class TestNodeFailure {
private static MiniOzoneCluster cluster; private static MiniOzoneCluster cluster;
private static OzoneConfiguration conf; private static OzoneConfiguration conf;
private static ContainerWithPipeline ratisContainer1; private static Pipeline ratisPipelineOne;
private static ContainerWithPipeline ratisContainer2; private static Pipeline ratisPipelineTwo;
private static ContainerManager containerManager; private static ContainerManager containerManager;
private static PipelineManager pipelineManager; private static PipelineManager pipelineManager;
private static long timeForFailure; private static long timeForFailure;
@ -76,10 +75,12 @@ public class TestNodeFailure {
StorageContainerManager scm = cluster.getStorageContainerManager(); StorageContainerManager scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager(); containerManager = scm.getContainerManager();
pipelineManager = scm.getPipelineManager(); pipelineManager = scm.getPipelineManager();
ratisContainer1 = containerManager.allocateContainer( ratisPipelineOne = pipelineManager.getPipeline(
RATIS, THREE, "testOwner"); containerManager.allocateContainer(
ratisContainer2 = containerManager.allocateContainer( RATIS, THREE, "testOwner").getPipelineID());
RATIS, THREE, "testOwner"); ratisPipelineTwo = pipelineManager.getPipeline(
containerManager.allocateContainer(
RATIS, THREE, "testOwner").getPipelineID());
// At this stage, there should be 2 pipeline one with 1 open container each. // At this stage, there should be 2 pipeline one with 1 open container each.
// Try closing the both the pipelines, one with a closed container and // Try closing the both the pipelines, one with a closed container and
// the other with an open container. // the other with an open container.
@ -99,12 +100,15 @@ public class TestNodeFailure {
} }
} }
@Ignore
// Enable this after we implement teardown pipeline logic once a datanode
// dies.
@Test(timeout = 300_000L) @Test(timeout = 300_000L)
public void testPipelineFail() throws InterruptedException, IOException, public void testPipelineFail() throws InterruptedException, IOException,
TimeoutException { TimeoutException {
Assert.assertEquals(ratisContainer1.getPipeline().getPipelineState(), Assert.assertEquals(ratisPipelineOne.getPipelineState(),
Pipeline.PipelineState.OPEN); Pipeline.PipelineState.OPEN);
Pipeline pipelineToFail = ratisContainer1.getPipeline(); Pipeline pipelineToFail = ratisPipelineOne;
DatanodeDetails dnToFail = pipelineToFail.getFirstNode(); DatanodeDetails dnToFail = pipelineToFail.getFirstNode();
cluster.shutdownHddsDatanode(dnToFail); cluster.shutdownHddsDatanode(dnToFail);
@ -112,18 +116,19 @@ public class TestNodeFailure {
Thread.sleep(3 * timeForFailure); Thread.sleep(3 * timeForFailure);
Assert.assertEquals(Pipeline.PipelineState.CLOSED, Assert.assertEquals(Pipeline.PipelineState.CLOSED,
pipelineManager.getPipeline(ratisContainer1.getPipeline().getId()) pipelineManager.getPipeline(ratisPipelineOne.getId())
.getPipelineState()); .getPipelineState());
Assert.assertEquals(Pipeline.PipelineState.OPEN, Assert.assertEquals(Pipeline.PipelineState.OPEN,
pipelineManager.getPipeline(ratisContainer2.getPipeline().getId()) pipelineManager.getPipeline(ratisPipelineTwo.getId())
.getPipelineState()); .getPipelineState());
// Now restart the datanode and make sure that a new pipeline is created. // Now restart the datanode and make sure that a new pipeline is created.
cluster.setWaitForClusterToBeReadyTimeout(300000); cluster.setWaitForClusterToBeReadyTimeout(300000);
cluster.restartHddsDatanode(dnToFail, true); cluster.restartHddsDatanode(dnToFail, true);
ContainerWithPipeline ratisContainer3 = Pipeline ratisPipelineThree = pipelineManager.getPipeline(
containerManager.allocateContainer(RATIS, THREE, "testOwner"); containerManager.allocateContainer(
RATIS, THREE, "testOwner").getPipelineID());
//Assert that new container is not created from the ratis 2 pipeline //Assert that new container is not created from the ratis 2 pipeline
Assert.assertNotEquals(ratisContainer3.getPipeline().getId(), Assert.assertNotEquals(ratisPipelineThree.getId(),
ratisContainer2.getPipeline().getId()); ratisPipelineTwo.getId());
} }
} }

View File

@ -21,6 +21,7 @@ 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.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.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline; .ContainerWithPipeline;
@ -62,10 +63,15 @@ public class TestPipelineClose {
cluster.waitForClusterToBeReady(); cluster.waitForClusterToBeReady();
scm = cluster.getStorageContainerManager(); scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager(); containerManager = scm.getContainerManager();
ratisContainer1 = containerManager pipelineManager = scm.getPipelineManager();
ContainerInfo containerInfo1 = containerManager
.allocateContainer(RATIS, THREE, "testOwner"); .allocateContainer(RATIS, THREE, "testOwner");
ratisContainer2 = containerManager ratisContainer1 = new ContainerWithPipeline(containerInfo1,
pipelineManager.getPipeline(containerInfo1.getPipelineID()));
ContainerInfo containerInfo2 = containerManager
.allocateContainer(RATIS, THREE, "testOwner"); .allocateContainer(RATIS, THREE, "testOwner");
ratisContainer2 = new ContainerWithPipeline(containerInfo2,
pipelineManager.getPipeline(containerInfo2.getPipelineID()));
pipelineManager = scm.getPipelineManager(); pipelineManager = scm.getPipelineManager();
// At this stage, there should be 2 pipeline one with 1 open container each. // At this stage, there should be 2 pipeline one with 1 open container each.
// Try closing the both the pipelines, one with a closed container and // Try closing the both the pipelines, one with a closed container and

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm.pipeline; package org.apache.hadoop.hdds.scm.pipeline;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneCluster;
@ -65,10 +66,12 @@ public class TestSCMRestart {
StorageContainerManager scm = cluster.getStorageContainerManager(); StorageContainerManager scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager(); containerManager = scm.getContainerManager();
pipelineManager = scm.getPipelineManager(); pipelineManager = scm.getPipelineManager();
ratisPipeline1 = containerManager.allocateContainer( ratisPipeline1 = pipelineManager.getPipeline(
RATIS, THREE, "Owner1").getPipeline(); containerManager.allocateContainer(
ratisPipeline2 = containerManager.allocateContainer( RATIS, THREE, "Owner1").getPipelineID());
RATIS, ONE, "Owner2").getPipeline(); ratisPipeline2 = pipelineManager.getPipeline(
containerManager.allocateContainer(
RATIS, ONE, "Owner2").getPipelineID());
// At this stage, there should be 2 pipeline one with 1 open container // At this stage, there should be 2 pipeline one with 1 open container
// each. Try restarting the SCM and then discover that pipeline are in // each. Try restarting the SCM and then discover that pipeline are in
// correct state. // correct state.
@ -100,10 +103,10 @@ public class TestSCMRestart {
Assert.assertEquals(ratisPipeline1AfterRestart, ratisPipeline1); Assert.assertEquals(ratisPipeline1AfterRestart, ratisPipeline1);
Assert.assertEquals(ratisPipeline2AfterRestart, ratisPipeline2); Assert.assertEquals(ratisPipeline2AfterRestart, ratisPipeline2);
// Try creating a new ratis pipeline, it should be from the same pipeline // Try creating a new container, it should be from the same pipeline
// as was before restart // as was before restart
Pipeline newRatisPipeline = newContainerManager ContainerInfo containerInfo = newContainerManager
.allocateContainer(RATIS, THREE, "Owner1").getPipeline(); .allocateContainer(RATIS, THREE, "Owner1");
Assert.assertEquals(newRatisPipeline.getId(), ratisPipeline1.getId()); Assert.assertEquals(containerInfo.getPipelineID(), ratisPipeline1.getId());
} }
} }

View File

@ -52,9 +52,8 @@ public class OzoneTestUtils {
.updateContainerState(ContainerID.valueof(blockID.getContainerID()), .updateContainerState(ContainerID.valueof(blockID.getContainerID()),
HddsProtos.LifeCycleEvent.CLOSE); HddsProtos.LifeCycleEvent.CLOSE);
Assert.assertFalse(scm.getContainerManager() Assert.assertFalse(scm.getContainerManager()
.getContainerWithPipeline(ContainerID.valueof( .getContainer(ContainerID.valueof(
blockID.getContainerID())) blockID.getContainerID())).isOpen());
.getContainerInfo().isOpen());
} catch (IOException e) { } catch (IOException e) {
e.printStackTrace(); e.printStackTrace();
} }

View File

@ -446,9 +446,10 @@ public class TestOzoneRestClient {
// Sum the data size from chunks in Container via containerID // Sum the data size from chunks in Container via containerID
// and localID, make sure the size equals to the actually value size. // and localID, make sure the size equals to the actually value size.
Pipeline pipeline = cluster.getStorageContainerManager() Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline( .getPipelineManager().getPipeline(
ContainerID.valueof(containerID)) cluster.getStorageContainerManager()
.getPipeline(); .getContainerManager().getContainer(
ContainerID.valueof(containerID)).getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1); Assert.assertEquals(datanodes.size(), 1);

View File

@ -23,6 +23,7 @@ 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.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.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
@ -377,10 +378,12 @@ public class TestCloseContainerHandlingByClient {
cluster.getStorageContainerManager().getEventQueue() cluster.getStorageContainerManager().getEventQueue()
.fireEvent(SCMEvents.CLOSE_CONTAINER, .fireEvent(SCMEvents.CLOSE_CONTAINER,
ContainerID.valueof(containerID)); ContainerID.valueof(containerID));
ContainerInfo container = cluster.getStorageContainerManager()
.getContainerManager()
.getContainer(ContainerID.valueof(containerID));
Pipeline pipeline = Pipeline pipeline =
cluster.getStorageContainerManager().getContainerManager() cluster.getStorageContainerManager().getPipelineManager()
.getContainerWithPipeline(ContainerID.valueof(containerID)) .getPipeline(container.getPipelineID());
.getPipeline();
pipelineList.add(pipeline); pipelineList.add(pipeline);
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
for (DatanodeDetails details : datanodes) { for (DatanodeDetails details : datanodes) {
@ -435,10 +438,13 @@ public class TestCloseContainerHandlingByClient {
List<OmKeyLocationInfo> locationInfos = List<OmKeyLocationInfo> locationInfos =
new ArrayList<>(groupOutputStream.getLocationInfoList()); new ArrayList<>(groupOutputStream.getLocationInfoList());
long containerID = locationInfos.get(0).getContainerID(); long containerID = locationInfos.get(0).getContainerID();
List<DatanodeDetails> datanodes = ContainerInfo container = cluster.getStorageContainerManager()
cluster.getStorageContainerManager().getContainerManager() .getContainerManager()
.getContainerWithPipeline(ContainerID.valueof(containerID)) .getContainer(ContainerID.valueof(containerID));
.getPipeline().getNodes(); Pipeline pipeline =
cluster.getStorageContainerManager().getPipelineManager()
.getPipeline(container.getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(1, datanodes.size()); Assert.assertEquals(1, datanodes.size());
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE); waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
dataString = fixedLengthString(keyString, (1 * blockSize)); dataString = fixedLengthString(keyString, (1 * blockSize));
@ -538,10 +544,13 @@ public class TestCloseContainerHandlingByClient {
List<OmKeyLocationInfo> locationInfos = List<OmKeyLocationInfo> locationInfos =
groupOutputStream.getLocationInfoList(); groupOutputStream.getLocationInfoList();
long containerID = locationInfos.get(0).getContainerID(); long containerID = locationInfos.get(0).getContainerID();
List<DatanodeDetails> datanodes = ContainerInfo container = cluster.getStorageContainerManager()
cluster.getStorageContainerManager().getContainerManager() .getContainerManager()
.getContainerWithPipeline(ContainerID.valueof(containerID)) .getContainer(ContainerID.valueof(containerID));
.getPipeline().getNodes(); Pipeline pipeline =
cluster.getStorageContainerManager().getPipelineManager()
.getPipeline(container.getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(1, datanodes.size()); Assert.assertEquals(1, datanodes.size());
// move the container on the datanode to Closing state, this will ensure // move the container on the datanode to Closing state, this will ensure
// closing the key will hit BLOCK_NOT_COMMITTED_EXCEPTION while trying // closing the key will hit BLOCK_NOT_COMMITTED_EXCEPTION while trying

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientRatis; import org.apache.hadoop.hdds.scm.XceiverClientRatis;
import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.*; import org.apache.hadoop.ozone.*;
@ -645,10 +644,10 @@ public class TestOzoneRpcClient {
Assert Assert
.assertEquals(value.getBytes().length, keyLocations.get(0).getLength()); .assertEquals(value.getBytes().length, keyLocations.get(0).getLength());
ContainerWithPipeline container = ContainerInfo container = cluster.getStorageContainerManager()
cluster.getStorageContainerManager().getContainerManager() .getContainerManager().getContainer(ContainerID.valueof(containerID));
.getContainerWithPipeline(new ContainerID(containerID)); Pipeline pipeline = cluster.getStorageContainerManager()
Pipeline pipeline = container.getPipeline(); .getPipelineManager().getPipeline(container.getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
DatanodeDetails datanodeDetails = datanodes.get(0); DatanodeDetails datanodeDetails = datanodes.get(0);
@ -662,17 +661,17 @@ public class TestOzoneRpcClient {
// shutdown the datanode // shutdown the datanode
cluster.shutdownHddsDatanode(datanodeDetails); cluster.shutdownHddsDatanode(datanodeDetails);
Assert.assertTrue(container.getContainerInfo().getState() Assert.assertTrue(container.getState()
== HddsProtos.LifeCycleState.OPEN); == HddsProtos.LifeCycleState.OPEN);
// try to read, this shouls be successful // try to read, this shouls be successful
readKey(bucket, keyName, value); readKey(bucket, keyName, value);
Assert.assertTrue(container.getContainerInfo().getState() Assert.assertTrue(container.getState()
== HddsProtos.LifeCycleState.OPEN); == HddsProtos.LifeCycleState.OPEN);
// shutdown the second datanode // shutdown the second datanode
datanodeDetails = datanodes.get(1); datanodeDetails = datanodes.get(1);
cluster.shutdownHddsDatanode(datanodeDetails); cluster.shutdownHddsDatanode(datanodeDetails);
Assert.assertTrue(container.getContainerInfo().getState() Assert.assertTrue(container.getState()
== HddsProtos.LifeCycleState.OPEN); == HddsProtos.LifeCycleState.OPEN);
// the container is open and with loss of 2 nodes we still should be able // the container is open and with loss of 2 nodes we still should be able
@ -750,10 +749,10 @@ public class TestOzoneRpcClient {
// Second, sum the data size from chunks in Container via containerID // Second, sum the data size from chunks in Container via containerID
// and localID, make sure the size equals to the size from keyDetails. // and localID, make sure the size equals to the size from keyDetails.
ContainerInfo container = cluster.getStorageContainerManager()
.getContainerManager().getContainer(ContainerID.valueof(containerID));
Pipeline pipeline = cluster.getStorageContainerManager() Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline( .getPipelineManager().getPipeline(container.getPipelineID());
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1); Assert.assertEquals(datanodes.size(), 1);

View File

@ -23,6 +23,7 @@ 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.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerID;
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.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneCluster;
@ -102,10 +103,10 @@ public class TestCloseContainerByPipeline {
.get(0).getBlocksLatestVersionOnly().get(0); .get(0).getBlocksLatestVersionOnly().get(0);
long containerID = omKeyLocationInfo.getContainerID(); long containerID = omKeyLocationInfo.getContainerID();
ContainerInfo container = cluster.getStorageContainerManager()
.getContainerManager().getContainer(ContainerID.valueof(containerID));
Pipeline pipeline = cluster.getStorageContainerManager() Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline( .getPipelineManager().getPipeline(container.getPipelineID());
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1); Assert.assertEquals(datanodes.size(), 1);
@ -158,10 +159,10 @@ public class TestCloseContainerByPipeline {
.get(0).getBlocksLatestVersionOnly().get(0); .get(0).getBlocksLatestVersionOnly().get(0);
long containerID = omKeyLocationInfo.getContainerID(); long containerID = omKeyLocationInfo.getContainerID();
ContainerInfo container = cluster.getStorageContainerManager()
.getContainerManager().getContainer(ContainerID.valueof(containerID));
Pipeline pipeline = cluster.getStorageContainerManager() Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline( .getPipelineManager().getPipeline(container.getPipelineID());
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1); Assert.assertEquals(datanodes.size(), 1);
@ -216,10 +217,10 @@ public class TestCloseContainerByPipeline {
.get(0).getBlocksLatestVersionOnly().get(0); .get(0).getBlocksLatestVersionOnly().get(0);
long containerID = omKeyLocationInfo.getContainerID(); long containerID = omKeyLocationInfo.getContainerID();
ContainerInfo container = cluster.getStorageContainerManager()
.getContainerManager().getContainer(ContainerID.valueof(containerID));
Pipeline pipeline = cluster.getStorageContainerManager() Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline( .getPipelineManager().getPipeline(container.getPipelineID());
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getNodes(); List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(3, datanodes.size()); Assert.assertEquals(3, datanodes.size());

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
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.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.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
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;
@ -80,28 +81,30 @@ public class TestCloseContainerHandler {
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions() cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
.get(0).getBlocksLatestVersionOnly().get(0); .get(0).getBlocksLatestVersionOnly().get(0);
long containerID = omKeyLocationInfo.getContainerID(); ContainerID containerId = ContainerID.valueof(
omKeyLocationInfo.getContainerID());
ContainerInfo container = cluster.getStorageContainerManager()
.getContainerManager().getContainer(containerId);
Pipeline pipeline = cluster.getStorageContainerManager() Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline( .getPipelineManager().getPipeline(container.getPipelineID());
ContainerID.valueof(containerID))
.getPipeline();
Assert.assertFalse(isContainerClosed(cluster, containerID)); Assert.assertFalse(isContainerClosed(cluster, containerId.getId()));
DatanodeDetails datanodeDetails = DatanodeDetails datanodeDetails =
cluster.getHddsDatanodes().get(0).getDatanodeDetails(); cluster.getHddsDatanodes().get(0).getDatanodeDetails();
//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.getId(),
HddsProtos.ReplicationType.STAND_ALONE, pipeline.getId())); HddsProtos.ReplicationType.STAND_ALONE, pipeline.getId()));
GenericTestUtils.waitFor(() -> isContainerClosed(cluster, containerID), GenericTestUtils.waitFor(() ->
isContainerClosed(cluster, containerId.getId()),
500, 500,
5 * 1000); 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)); Assert.assertTrue(isContainerClosed(cluster, containerId.getId()));
} }
private Boolean isContainerClosed(MiniOzoneCluster cluster, private Boolean isContainerClosed(MiniOzoneCluster cluster,

View File

@ -97,6 +97,14 @@ public class TestFreonWithDatanodeFastRestart {
TermIndex termIndexAfterRestart = sm.getLastAppliedTermIndex(); TermIndex termIndexAfterRestart = sm.getLastAppliedTermIndex();
Assert.assertTrue(termIndexAfterRestart.getIndex() >= Assert.assertTrue(termIndexAfterRestart.getIndex() >=
termIndexBeforeRestart.getIndex()); termIndexBeforeRestart.getIndex());
// TODO: fix me
// Give some time for the datanode to register again with SCM.
// If we try to use the pipeline before the datanode registers with SCM
// we end up in "NullPointerException: scmId cannot be null" in
// datanode statemachine and datanode crashes.
// This has to be fixed. Check HDDS-830.
// Until then this sleep should help us!
Thread.sleep(5000);
startFreon(); startFreon();
} }

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -77,6 +78,10 @@ public class TestFreonWithDatanodeRestart {
} }
} }
@Ignore
// Pipeline close is not happening now, this requires HDDS-801 and
// pipeline teardown logic in place. Enable this once those things are in
// place
@Test @Test
public void testRestart() throws Exception { public void testRestart() throws Exception {
startFreon(); startFreon();

View File

@ -128,7 +128,7 @@ public class TestContainerSQLCli {
containerManager = new SCMContainerManager(conf, nodeManager, containerManager = new SCMContainerManager(conf, nodeManager,
pipelineManager, eventQueue); pipelineManager, eventQueue);
blockManager = new BlockManagerImpl( blockManager = new BlockManagerImpl(
conf, nodeManager, containerManager, eventQueue); conf, nodeManager, pipelineManager, containerManager, eventQueue);
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager); eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false); eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);
GenericTestUtils.waitFor(() -> { GenericTestUtils.waitFor(() -> {