diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java index e0b28548e8e4..5890c359fdb5 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java @@ -269,6 +269,11 @@ public final class ScmConfigKeys { public static final String OZONE_SCM_STALENODE_INTERVAL_DEFAULT = "5m"; + public static final String OZONE_SCM_PENDING_CONTAINER_ROLL_INTERVAL = + "ozone.scm.pending.container.roll.interval"; + public static final String OZONE_SCM_PENDING_CONTAINER_ROLL_INTERVAL_DEFAULT = + "5m"; + public static final String OZONE_SCM_HEARTBEAT_RPC_TIMEOUT = "ozone.scm.heartbeat.rpc-timeout"; public static final String OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT = diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 3089132ca0b8..44dbfef766c5 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -1037,6 +1037,17 @@ balances the amount of metadata. + + ozone.scm.pending.container.roll.interval + 5m + OZONE, SCM, PERFORMANCE, MANAGEMENT + + The interval at which the two-window tumbling bucket for pending + container allocations rolls over per DataNode. Pending containers + that have not been confirmed within two intervals are automatically + aged out. Default is 5 minutes. + + ozone.scm.container.lock.stripes 512 diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java index 8340480f1a4c..337f5b849ac3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java @@ -237,16 +237,17 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner) private ContainerInfo allocateContainer(final Pipeline pipeline, final String owner) throws IOException { - if (!pipelineManager.hasEnoughSpace(pipeline)) { - LOG.debug("Cannot allocate a new container because pipeline {} does not have enough space.", pipeline); - return null; - } - final long uniqueId = sequenceIdGen.getNextId(SequenceIdType.containerId); Preconditions.checkState(uniqueId > 0, "Cannot allocate container, negative container id" + " generated. %s.", uniqueId); final ContainerID containerID = ContainerID.valueOf(uniqueId); + + if (!pipelineManager.checkSpaceAndRecordAllocation(pipeline, containerID)) { + LOG.debug("Cannot allocate a new container because pipeline {} does not have enough space.", pipeline); + return null; + } + final ContainerInfoProto.Builder containerInfoBuilder = ContainerInfoProto .newBuilder() .setState(LifeCycleState.OPEN) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java index 0cebcb10ef2c..b1dc0abff667 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.scm.container.report.ContainerReportValidator; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.ha.SCMContext; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode; @@ -153,6 +154,7 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, containerReport.getReportsList(); final Set expectedContainersInDatanode = getNodeManager().getContainers(datanodeDetails); + DatanodeInfo datanodeInfo = datanodeDetails instanceof DatanodeInfo ? (DatanodeInfo) datanodeDetails : null; for (ContainerReplicaProto replica : replicas) { ContainerID cid = ContainerID.valueOf(replica.getContainerID()); @@ -175,6 +177,11 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, if (!alreadyInDn) { // This is a new Container not in the nodeManager -> dn map yet getNodeManager().addContainer(datanodeDetails, cid); + // Remove from pending tracker when container is added to DN + // This container was just confirmed for the first time on this DN + if (datanodeInfo != null) { + getNodeManager().removePendingAllocationForDatanode(datanodeInfo, cid); + } } if (container == null || ContainerReportValidator .validate(container, datanodeDetails, replica)) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java index 247e3667d9ef..06358c352cca 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.scm.container.report.ContainerReportValidator; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.ha.SCMContext; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.IncrementalContainerReportFromDatanode; @@ -83,6 +84,7 @@ protected void processICR(IncrementalContainerReportFromDatanode report, // issue between the container list in NodeManager and the replicas in // ContainerManager. synchronized (dd) { + DatanodeInfo datanodeInfo = dd instanceof DatanodeInfo ? (DatanodeInfo) dd : null; for (ContainerReplicaProto replicaProto : report.getReport().getReportList()) { Object detailsForLogging = getDetailsForLogging(null, replicaProto, dd); @@ -103,6 +105,9 @@ protected void processICR(IncrementalContainerReportFromDatanode report, } if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); + if (datanodeInfo != null) { + getNodeManager().removePendingAllocationForDatanode(datanodeInfo, id); + } } success = true; } catch (ContainerNotFoundException e) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java index 4fb7f84394f3..6a2d92b64a5e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java @@ -185,17 +185,23 @@ default int getAllNodeCount() { DatanodeInfo getDatanodeInfo(DatanodeDetails dn); /** - * True if the node can accept another container of the given size. + * Atomically checks if the datanode has space for a new container and records the allocation + * if space is available. This prevents race conditions where multiple threads check space + * concurrently and over-allocate. + * + * @param datanodeInfo node info of the receiving the allocation + * @param containerID the container being allocated + * @return true if space was available and allocation was recorded, false otherwise */ - boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID); + boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID); /** - * Records a pending container allocation for a single DataNode identified by its ID. + * Removes a pending container allocation from a datanode. * - * @param datanodeID the ID of the DataNode receiving the allocation - * @param containerID the container being allocated + * @param datanodeInfo info about the datanode + * @param containerID the container to remove from pending */ - void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID); + void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID); /** * Return the node stat of the specified datanode. @@ -435,4 +441,9 @@ default void removeNode(DatanodeDetails datanodeDetails) throws NodeNotFoundExce } int openContainerLimit(List datanodes); + + /** + * SCM-side tracker for container allocations not yet reported by datanodes. + */ + PendingContainerTracker getPendingContainerTracker(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java index 9e4b96999df0..57ee8ef9adb6 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.LayoutVersionProto; +import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.ha.SCMContext; @@ -124,7 +125,7 @@ public class NodeStateManager implements Runnable, Closeable { */ private final long deadNodeIntervalMs; - private final long containerRollIntervalMs = 5 * 60 * 1000; //TODO + private final long containerRollIntervalMs; /** * The future is used to pause/unpause the scheduled checks. @@ -214,6 +215,11 @@ public NodeStateManager(ConfigurationSource conf, scmContext.getFinalizationCheckpoint()) && !layoutMatchCondition.test(layout); + containerRollIntervalMs = conf.getTimeDuration( + ScmConfigKeys.OZONE_SCM_PENDING_CONTAINER_ROLL_INTERVAL, + ScmConfigKeys.OZONE_SCM_PENDING_CONTAINER_ROLL_INTERVAL_DEFAULT, + TimeUnit.MILLISECONDS); + scheduleNextHealthCheck(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java index fc7bbc238192..3821727ed971 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java @@ -17,6 +17,7 @@ package org.apache.hadoop.hdds.scm.node; +import com.google.common.annotations.VisibleForTesting; import java.util.HashSet; import java.util.List; import java.util.Objects; @@ -120,16 +121,6 @@ synchronized boolean contains(ContainerID containerID) { return currentWindow.contains(containerID) || previousWindow.contains(containerID); } - /** - * Add container to current window. - */ - synchronized boolean add(ContainerID containerID) { - boolean added = currentWindow.add(containerID); - LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", - containerID, datanodeID, added, getCount()); - return added; - } - /** * Remove container from both windows. */ @@ -148,6 +139,34 @@ synchronized boolean remove(ContainerID containerID) { synchronized int getCount() { return currentWindow.size() + previousWindow.size(); } + + /** + * Atomically checks whether there is allocatable space for one more container of + * {@code maxContainerSize} given the current pending count, and adds {@code containerID} + * to the current window if so. + * + * @param storageReports storage reports for the datanode + * @param maxContainerSize maximum size of a single container in bytes + * @param containerID the container being allocated + * @return true if space was available and the container was recorded, false otherwise + */ + + synchronized boolean checkSpaceAndAdd( + List storageReports, long maxContainerSize, ContainerID containerID) { + final int pendingAllocationCount = getCount(); + long allocatableCount = 0; + for (StorageReportProto report : storageReports) { + final long allocatableCountOnThisDisk = VolumeUsage.getUsableSpace(report) / maxContainerSize; + allocatableCount += allocatableCountOnThisDisk; + if (allocatableCount > pendingAllocationCount) { + final boolean added = currentWindow.add(containerID); + LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", + containerID, datanodeID, added, getCount()); + return added; + } + } + return false; + } } public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, SCMNodeMetrics metrics) { @@ -158,52 +177,34 @@ public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, SCMNo } /** - * Whether the datanode can fit another container of {@link #maxContainerSize} after accounting for - * SCM pending allocations for {@code node} (this tracker) and usable space across volumes on - * {@code datanodeInfo}. Pending bytes are count × {@code maxContainerSize}; - * effective allocatable space sums full-container slots per storage report. + * Atomically checks if the datanode has space for a new container and records the allocation + * if space is available. The check-and-add atomicity is enforced inside + * {@link TwoWindowBucket#checkSpaceAndAdd}. * - * @param datanodeInfo storage reports for the datanode + * @param datanodeInfo datanode whose storage reports and pending bucket + * @param containerID the container being allocated + * @return true if space was available and the allocation was recorded, false otherwise */ - public boolean hasEffectiveAllocatableSpaceForNewContainer(DatanodeInfo datanodeInfo) { + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { Objects.requireNonNull(datanodeInfo, "datanodeInfo == null"); + Objects.requireNonNull(containerID, "containerID == null"); - long pendingAllocationSize = datanodeInfo.getPendingContainerAllocations().getCount() * maxContainerSize; List storageReports = datanodeInfo.getStorageReports(); Objects.requireNonNull(storageReports, "storageReports == null"); if (storageReports.isEmpty()) { return false; } - long effectiveAllocatableSpace = 0L; - for (StorageReportProto report : storageReports) { - long usableSpace = VolumeUsage.getUsableSpace(report); - long containersOnThisDisk = usableSpace / maxContainerSize; - effectiveAllocatableSpace += containersOnThisDisk * maxContainerSize; - if (effectiveAllocatableSpace - pendingAllocationSize >= maxContainerSize) { - return true; - } - } - if (metrics != null) { - metrics.incNumSkippedFullNodeContainerAllocation(); - } - return false; - } - /** - * Record a pending container allocation for a single DataNode. - * Container is added to the current window. - * - * @param containerID The container being allocated/replicated - */ - public void recordPendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) { - Objects.requireNonNull(containerID, "containerID == null"); - if (datanodeInfo == null) { - return; - } - final boolean added = datanodeInfo.getPendingContainerAllocations().add(containerID); - if (added && metrics != null) { - metrics.incNumPendingContainersAdded(); + boolean added = datanodeInfo.getPendingContainerAllocations() + .checkSpaceAndAdd(storageReports, maxContainerSize, containerID); + if (metrics != null) { + if (added) { + metrics.incNumPendingContainersAdded(); + } else { + metrics.incNumSkippedFullNodeContainerAllocation(); + } } + return added; } /** @@ -222,4 +223,9 @@ public void removePendingAllocation(TwoWindowBucket bucket, ContainerID containe metrics.incNumPendingContainersRemoved(); } } + + @VisibleForTesting + public SCMNodeMetrics getMetrics() { + return metrics; + } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index ad392a247d53..c53c7df4cb35 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -192,7 +192,10 @@ public SCMNodeManager( this.pendingContainerTracker = new PendingContainerTracker( (long) conf.getStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES), - 5 * 60 * 1000, // TODO + conf.getTimeDuration( + ScmConfigKeys.OZONE_SCM_PENDING_CONTAINER_ROLL_INTERVAL, + ScmConfigKeys.OZONE_SCM_PENDING_CONTAINER_ROLL_INTERVAL_DEFAULT, + TimeUnit.MILLISECONDS), this.metrics); this.clusterMap = networkTopology; this.nodeResolver = nodeResolver; @@ -231,6 +234,11 @@ private void unregisterMXBean() { } } + @Override + public PendingContainerTracker getPendingContainerTracker() { + return pendingContainerTracker; + } + protected NodeStateManager getNodeStateManager() { return nodeStateManager; } @@ -1071,28 +1079,15 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dn) { } } - /** - * Effective space check aligned with container allocation: per-disk slot model minus - * SCM pending allocations. - */ @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID) { - DatanodeInfo datanodeInfo = getNode(datanodeID); - if (datanodeInfo == null) { - LOG.warn("DatanodeInfo not found for node {}", datanodeID); - return false; - } - return pendingContainerTracker.hasEffectiveAllocatableSpaceForNewContainer(datanodeInfo); + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { + return pendingContainerTracker.checkSpaceAndRecordAllocation(datanodeInfo, containerID); } @Override - public void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { - DatanodeInfo datanodeInfo = getNode(datanodeID); - if (datanodeInfo == null) { - LOG.warn("DatanodeInfo not found for node {}", datanodeID); - return; - } - pendingContainerTracker.recordPendingAllocationForDatanode(datanodeInfo, containerID); + public void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) { + pendingContainerTracker.removePendingAllocation( + datanodeInfo.getPendingContainerAllocations(), containerID); } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java index 0014936a80db..0b90247ae8ab 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java @@ -136,6 +136,14 @@ void incNumPendingContainersRemoved() { numPendingContainersRemoved.incr(); } + public long getNumPendingContainersAdded() { + return numPendingContainersAdded.value(); + } + + public long getNumPendingContainersRemoved() { + return numPendingContainersRemoved.value(); + } + void incNumSkippedFullNodeContainerAllocation() { numSkippedFullNodeContainerAllocation.incr(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java index 739b0c058ec8..e0d9de1f10b5 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java @@ -94,17 +94,6 @@ int getPipelineCount( void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID) throws PipelineNotFoundException, InvalidPipelineStateException; - /** - * Records a pending container allocation for every DataNode in the pipeline. - * The allocation is tracked in each node's two-window tumbling bucket so that - * {@code hasEnoughSpace} can account for in-flight allocations before a container - * report arrives from the DataNode. - * - * @param pipeline the pipeline whose nodes will receive the pending record - * @param containerID the container being allocated - */ - void recordPendingAllocation(Pipeline pipeline, ContainerID containerID); - /** * Add container to pipeline during SCM Start. * @@ -224,13 +213,15 @@ void reinitialize(Table pipelineStore) void releaseWriteLock(); /** - * Checks whether all Datanodes in the specified pipeline have enough space to store a new container. + * Atomically checks if all datanodes in the pipeline have space for a new container + * and records the allocation if space is available. This prevents race conditions + * where multiple threads check space concurrently and over-allocate. * - * @param pipeline pipeline to check - * @return false if any Datanode in the pipeline has no volume with space greater than the configured - * container size, otherwise true + * @param pipeline the pipeline whose nodes will be checked and recorded + * @param containerID the container being allocated + * @return true if all nodes had space and allocation was recorded, false otherwise */ - boolean hasEnoughSpace(Pipeline pipeline); + boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID); int openContainerLimit(List datanodes); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index c4375e3f20ea..bdb286292d10 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java @@ -23,6 +23,7 @@ import java.time.Clock; import java.time.Duration; import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -53,6 +54,7 @@ import org.apache.hadoop.hdds.scm.ha.SCMContext; import org.apache.hadoop.hdds.scm.ha.SCMHAManager; import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.server.upgrade.FinalizationManager; import org.apache.hadoop.hdds.server.events.EventPublisher; @@ -634,20 +636,29 @@ private boolean isOpenWithUnregisteredNodes(Pipeline pipeline) { } @Override - public boolean hasEnoughSpace(Pipeline pipeline) { - for (DatanodeDetails node : pipeline.getNodes()) { - if (!nodeManager.hasSpaceForNewContainerAllocation(node.getID())) { + public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { + final Set datanodeDetails = pipeline.getNodeSet(); + final List datanodeInfos = new ArrayList<>(datanodeDetails.size()); + for (DatanodeDetails dn : datanodeDetails) { + final DatanodeInfo info = nodeManager.getDatanodeInfo(dn); + if (info == null) { + LOG.warn("DatanodeInfo not found for {}", dn.getID()); return false; } + datanodeInfos.add(info); } - return true; - } - @Override - public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { - for (DatanodeDetails dn : pipeline.getNodes()) { - nodeManager.recordPendingAllocationForDatanode(dn.getID(), containerID); + final List successfulNodes = new ArrayList<>(datanodeInfos.size()); + for (DatanodeInfo dn : datanodeInfos) { + if (!nodeManager.checkSpaceAndRecordAllocation(dn, containerID)) { + for (DatanodeInfo rollbackNode : successfulNodes) { + nodeManager.removePendingAllocationForDatanode(rollbackNode, containerID); + } + return false; + } + successfulNodes.add(dn); } + return true; } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index cb4bb156769c..010e0ca8b6c4 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -256,6 +256,13 @@ public ContainerWithPipeline allocateContainer(ReplicationConfig replicationConf getScm().checkAdminAccess(getRemoteUser(), false); final ContainerInfo container = scm.getContainerManager() .allocateContainer(replicationConfig, owner); + if (container == null) { + throw new SCMException( + "Could not allocate container for replication " + replicationConfig + + ", owner=" + owner + + ": no suitable open pipeline with enough space", + ResultCodes.FAILED_TO_ALLOCATE_CONTAINER); + } final Pipeline pipeline = scm.getPipelineManager() .getPipeline(container.getPipelineID()); ContainerWithPipeline cp = new ContainerWithPipeline(container, pipeline); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java index 57d38ece3dd6..1b8c4bbf384d 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java @@ -454,12 +454,19 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dd) { } @Override - public void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { - DatanodeDetails dd = nodeMetricMap.keySet().stream() - .filter(d -> d.getID().equals(datanodeID)) - .findFirst().orElse(null); - DatanodeInfo info = getDatanodeInfo(dd); - pendingContainerTracker.recordPendingAllocationForDatanode(info, containerID); + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { + if (datanodeInfo == null) { + return false; + } + return pendingContainerTracker.checkSpaceAndRecordAllocation(datanodeInfo, containerID); + } + + @Override + public void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) { + if (datanodeInfo != null) { + pendingContainerTracker.removePendingAllocation( + datanodeInfo.getPendingContainerAllocations(), containerID); + } } /** @@ -943,6 +950,11 @@ public int openContainerLimit(List datanodes) { return 9; } + @Override + public PendingContainerTracker getPendingContainerTracker() { + return pendingContainerTracker; + } + @Override public long getLastHeartbeat(DatanodeDetails datanodeDetails) { return -1; @@ -956,18 +968,6 @@ public void setNumHealthyVolumes(int value) { numHealthyDisksPerDatanode = value; } - @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID) { - DatanodeDetails dd = nodeMetricMap.keySet().stream() - .filter(d -> d.getID().equals(datanodeID)) - .findFirst().orElse(null); - DatanodeInfo info = getDatanodeInfo(dd); - if (info == null) { - return false; - } - return pendingContainerTracker.hasEffectiveAllocatableSpaceForNewContainer(info); - } - /** * A class to declare some values for the nodes so that our tests * won't fail. diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/SimpleMockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/SimpleMockNodeManager.java index f2da8fd2878b..e47b979b4750 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/SimpleMockNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/SimpleMockNodeManager.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.node.NodeStatus; +import org.apache.hadoop.hdds.scm.node.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; @@ -63,6 +64,7 @@ public class SimpleMockNodeManager implements NodeManager { private Map nodeMap = new ConcurrentHashMap<>(); private Map> pipelineMap = new ConcurrentHashMap<>(); private Map> containerMap = new ConcurrentHashMap<>(); + private PendingContainerTracker pendingContainerTracker; public void register(DatanodeDetails dd, NodeStatus status) { dd.setPersistedOpState(status.getOperationalState()); @@ -250,12 +252,12 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dn) { } @Override - public void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { + return true; } @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID) { - return true; + public void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) { } @Override @@ -445,4 +447,12 @@ public Boolean isNodeRegistered(DatanodeDetails datanodeDetails) { return false; } + @Override + public PendingContainerTracker getPendingContainerTracker() { + int rollIntervalMs = 5 * 60 * 1000; + if (pendingContainerTracker == null) { + pendingContainerTracker = new PendingContainerTracker(5L * 1024 * 1024 * 1024, rollIntervalMs, null); + } + return pendingContainerTracker; + } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java index 68dcc634a5e3..c64db89290aa 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java @@ -102,7 +102,9 @@ void setUp() throws Exception { pipelineManager = spy(base); // Default: allow allocation in tests unless a test overrides it. - doReturn(true).when(pipelineManager).hasEnoughSpace(any(Pipeline.class)); + // Allocation uses checkSpaceAndRecordAllocation + doReturn(true).when(pipelineManager) + .checkSpaceAndRecordAllocation(any(Pipeline.class), any(ContainerID.class)); pipelineManager.createPipeline(RatisReplicationConfig.getInstance( ReplicationFactor.THREE)); @@ -140,11 +142,12 @@ void testAllocateContainer() throws Exception { */ @Test public void testGetMatchingContainerReturnsNullWhenNotEnoughSpaceInDatanodes() throws IOException { - doReturn(false).when(pipelineManager).hasEnoughSpace(any()); + doReturn(false).when(pipelineManager) + .checkSpaceAndRecordAllocation(any(Pipeline.class), any(ContainerID.class)); long sizeRequired = 256 * 1024 * 1024; // 256 MB Pipeline pipeline = pipelineManager.getPipelines().iterator().next(); - // MockPipelineManager#hasEnoughSpace always returns false + // MockPipelineManager#checkSpaceAndRecordAllocation always returns false // the pipeline has no existing containers, so a new container gets allocated in getMatchingContainer ContainerInfo container = containerManager .getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); @@ -162,10 +165,10 @@ public void testGetMatchingContainerReturnsNullWhenNotEnoughSpaceInDatanodes() t public void testGetMatchingContainerReturnsContainerWhenEnoughSpaceInDatanodes() throws IOException { long sizeRequired = 256 * 1024 * 1024; // 256 MB - // create a spy to mock hasEnoughSpace to always return true + // create a spy to mock checkSpaceAndRecordAllocation to always return true PipelineManager spyPipelineManager = spy(pipelineManager); doReturn(true).when(spyPipelineManager) - .hasEnoughSpace(any(Pipeline.class)); + .checkSpaceAndRecordAllocation(any(Pipeline.class), any(ContainerID.class)); // create a new ContainerManager using the spy File tempDir = new File(testDir, "tempDir"); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java index 4dbe79fc1351..681344531fa0 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hdds.scm.PlacementPolicy; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.XceiverClientManager; +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.ContainerManagerImpl; @@ -69,6 +70,7 @@ import org.apache.hadoop.hdds.scm.net.NodeSchemaManager; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.server.SCMStorageConfig; import org.apache.hadoop.hdds.server.events.EventQueue; @@ -159,7 +161,8 @@ SCMNodeManager createNodeManager(OzoneConfiguration config) { ContainerManager createContainerManager() throws IOException { pipelineManager = spy(pipelineManager); - doReturn(true).when(pipelineManager).hasEnoughSpace(any()); + doReturn(true).when(pipelineManager) + .checkSpaceAndRecordAllocation(any(Pipeline.class), any(ContainerID.class)); return new ContainerManagerImpl(conf, scmhaManager, sequenceIdGen, pipelineManager, diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestPendingContainerTracker.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestPendingContainerTracker.java index c747dc7d60ae..b486715deb01 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestPendingContainerTracker.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestPendingContainerTracker.java @@ -57,9 +57,11 @@ public void setUp() throws IOException { datanodes = new ArrayList<>(NUM_DATANODES); for (int i = 0; i < NUM_DATANODES; i++) { - datanodes.add(new DatanodeInfo( + DatanodeInfo dn = new DatanodeInfo( MockDatanodeDetails.randomLocalDatanodeDetails(), NodeStatus.inServiceHealthy(), null, - HddsTestUtils.ROLL_INTERVAL_MS_DEFAULT)); + HddsTestUtils.ROLL_INTERVAL_MS_DEFAULT); + setupDefaultStorageReport(dn); + datanodes.add(dn); } containers = new ArrayList<>(NUM_CONTAINERS); @@ -74,11 +76,17 @@ public void setUp() throws IOException { container2 = containers.get(1); } + private void setupDefaultStorageReport(DatanodeInfo dn) { + List reports = new ArrayList<>(); + reports.add(createStorageReport(dn, 10_000 * MAX_CONTAINER_SIZE, 10_000 * MAX_CONTAINER_SIZE, 0)); + dn.updateStorageReports(reports); + } + @Test public void testRecordPendingAllocation() { // Allocate first 100 containers, one per datanode for (int i = 0; i < 100; i++) { - tracker.recordPendingAllocationForDatanode(datanodes.get(i), containers.get(i)); + tracker.checkSpaceAndRecordAllocation(datanodes.get(i), containers.get(i)); } // Each of the first 100 DNs should have 1 pending container @@ -97,7 +105,7 @@ public void testRecordPendingAllocation() { public void testRemovePendingAllocation() { // Allocate containers 0-99, one per datanode for (int i = 0; i < 100; i++) { - tracker.recordPendingAllocationForDatanode(datanodes.get(i), containers.get(i)); + tracker.checkSpaceAndRecordAllocation(datanodes.get(i), containers.get(i)); } // Remove from first 50 DNs @@ -130,8 +138,9 @@ public void testTwoWindowRollAgesOutContainerAfterTwoIntervals() throws Interrup rollMs); PendingContainerTracker shortRollTracker = new PendingContainerTracker(MAX_CONTAINER_SIZE, rollMs, null); + setupDefaultStorageReport(shortDn); - shortRollTracker.recordPendingAllocationForDatanode(shortDn, container1); + shortRollTracker.checkSpaceAndRecordAllocation(shortDn, container1); assertEquals(1, shortDn.getPendingContainerAllocations().getCount()); assertTrue(shortDn.getPendingContainerAllocations().contains(container1)); @@ -150,7 +159,7 @@ public void testTwoWindowRollAgesOutContainerAfterTwoIntervals() throws Interrup @Test public void testRemoveNonExistentContainer() { - datanodes.subList(0, 3).forEach(dn -> tracker.recordPendingAllocationForDatanode(dn, container1)); + datanodes.subList(0, 3).forEach(dn -> tracker.checkSpaceAndRecordAllocation(dn, container1)); // Remove a container that was never added - should not throw exception tracker.removePendingAllocation(dn1.getPendingContainerAllocations(), container2); @@ -180,7 +189,7 @@ public void testConcurrentModification() throws InterruptedException { threads[i] = new Thread(() -> { for (int j = 0; j < operationsPerThread; j++) { ContainerID cid = ContainerID.valueOf(threadId * 1000L + j); - datanodes.subList(0, 3).forEach(dn -> tracker.recordPendingAllocationForDatanode(dn, cid)); + datanodes.subList(0, 3).forEach(dn -> tracker.checkSpaceAndRecordAllocation(dn, cid)); if (j % 2 == 0) { tracker.removePendingAllocation(dn1.getPendingContainerAllocations(), cid); @@ -202,7 +211,7 @@ public void testConcurrentModification() throws InterruptedException { @Test public void testBucketsRetainedWhenEmpty() { - datanodes.subList(0, 3).forEach(dn -> tracker.recordPendingAllocationForDatanode(dn, container1)); + datanodes.subList(0, 3).forEach(dn -> tracker.checkSpaceAndRecordAllocation(dn, container1)); assertEquals(1, dn1.getPendingContainerAllocations().getCount()); @@ -213,7 +222,7 @@ public void testBucketsRetainedWhenEmpty() { assertEquals(1, dn2.getPendingContainerAllocations().getCount()); // Empty bucket for DN1 is still usable for new allocations - tracker.recordPendingAllocationForDatanode(dn1, container2); + tracker.checkSpaceAndRecordAllocation(dn1, container2); assertEquals(1, dn1.getPendingContainerAllocations().getCount()); } @@ -223,8 +232,8 @@ public void testRemoveFromBothWindows() { // In general, a container could be in previous window after a roll // Add containers - datanodes.subList(0, 3).forEach(dn -> tracker.recordPendingAllocationForDatanode(dn, container1)); - datanodes.subList(0, 3).forEach(dn -> tracker.recordPendingAllocationForDatanode(dn, container2)); + datanodes.subList(0, 3).forEach(dn -> tracker.checkSpaceAndRecordAllocation(dn, container1)); + datanodes.subList(0, 3).forEach(dn -> tracker.checkSpaceAndRecordAllocation(dn, container2)); assertEquals(2, dn1.getPendingContainerAllocations().getCount()); @@ -242,7 +251,7 @@ public void testManyContainersOnSingleDatanode() { // Allocate first 1000 containers to the first datanode DatanodeInfo dn = datanodes.get(0); for (int i = 0; i < 1000; i++) { - tracker.recordPendingAllocationForDatanode(dn, containers.get(i)); + tracker.checkSpaceAndRecordAllocation(dn, containers.get(i)); } assertEquals(1000, dn.getPendingContainerAllocations().getCount()); @@ -270,7 +279,7 @@ public void testAllDatanodesWithMultipleContainers() { DatanodeInfo dn = datanodes.get(dnIdx); for (int cIdx = 0; cIdx < 10; cIdx++) { int containerIdx = dnIdx * 10 + cIdx; - tracker.recordPendingAllocationForDatanode(dn, containers.get(containerIdx)); + tracker.checkSpaceAndRecordAllocation(dn, containers.get(containerIdx)); } } @@ -308,7 +317,7 @@ public void testIdempotentRecording() { for (int round = 0; round < 5; round++) { for (int i = 0; i < 100; i++) { - tracker.recordPendingAllocationForDatanode(dn, containers.get(i)); + tracker.checkSpaceAndRecordAllocation(dn, containers.get(i)); } } @@ -320,7 +329,6 @@ public void testIdempotentRecording() { public void testMultiVolumeAccumulatedSpaceIsNotEnough() { long containerSize = MAX_CONTAINER_SIZE; - // Use the same DatanodeInfo object for both recording and checking. DatanodeInfo dnInfo = datanodes.get(0); List reports = new ArrayList<>(); reports.add(createStorageReport(dnInfo, 100 * containerSize, containerSize / 4, 0)); @@ -328,51 +336,44 @@ public void testMultiVolumeAccumulatedSpaceIsNotEnough() { reports.add(createStorageReport(dnInfo, 100 * containerSize, containerSize / 2, 0)); dnInfo.updateStorageReports(reports); - assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dnInfo)); + assertFalse(tracker.checkSpaceAndRecordAllocation(dnInfo, containers.get(0))); } @Test public void testMultiVolumeWithPendingAllocation() { long containerSize = MAX_CONTAINER_SIZE; - // Use the same DatanodeInfo object for recording pending allocations and checking space. DatanodeInfo dnInfo = datanodes.get(0); - // Remaining space available for 3 containers across all the volumes - tracker.recordPendingAllocationForDatanode(dnInfo, containers.get(0)); - tracker.recordPendingAllocationForDatanode(dnInfo, containers.get(1)); - + // 3 volumes × 1 slot each = 3 total slots List reports = new ArrayList<>(); reports.add(createStorageReport(dnInfo, 100 * containerSize, containerSize, 0)); reports.add(createStorageReport(dnInfo, 50 * containerSize, containerSize, 0)); reports.add(createStorageReport(dnInfo, 100 * containerSize, containerSize, 0)); dnInfo.updateStorageReports(reports); - // Remaining space available for 1 container across all the volume after 2 container allocation - - assertTrue(tracker.hasEffectiveAllocatableSpaceForNewContainer(dnInfo)); - tracker.recordPendingAllocationForDatanode(dnInfo, containers.get(2)); - // Remaining space available for 0 container across all the volume - assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dnInfo)); + // Record 3 allocations atomically, each should succeed + assertTrue(tracker.checkSpaceAndRecordAllocation(dnInfo, containers.get(0))); + assertTrue(tracker.checkSpaceAndRecordAllocation(dnInfo, containers.get(1))); + assertTrue(tracker.checkSpaceAndRecordAllocation(dnInfo, containers.get(2))); + // All 3 slots consumed, 4th allocation must fail + assertFalse(tracker.checkSpaceAndRecordAllocation(dnInfo, containers.get(3))); } @Test public void testMultiVolumeWithCommittedBytes() { long containerSize = MAX_CONTAINER_SIZE; - // Use the same DatanodeInfo object for recording pending allocations and checking space. DatanodeInfo dnInfo = datanodes.get(0); List reports = new ArrayList<>(); reports.add(createStorageReport(dnInfo, 100 * containerSize, 6 * containerSize, 5 * containerSize)); reports.add(createStorageReport(dnInfo, 50 * containerSize, 3 * containerSize, 3 * containerSize)); dnInfo.updateStorageReports(reports); - // Remaining space available for 1 container across all the volume considering committed bytes - assertTrue(tracker.hasEffectiveAllocatableSpaceForNewContainer(dnInfo)); - tracker.recordPendingAllocationForDatanode(dnInfo, containers.get(0)); - // Remaining space available for 0 container across all the volume considering - // committed bytes and container allocation - assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dnInfo)); + // 1 slot available — first allocation succeeds and consumes it + assertTrue(tracker.checkSpaceAndRecordAllocation(dnInfo, containers.get(0))); + // 0 slots remaining + assertFalse(tracker.checkSpaceAndRecordAllocation(dnInfo, containers.get(1))); } private StorageReportProto createStorageReport(DatanodeInfo dn, long capacity, long remaining, long committed) { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java index 69b1e24dfe3b..229d9283c74b 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java @@ -334,15 +334,13 @@ public boolean isPipelineCreationFrozen() { } @Override - public boolean hasEnoughSpace(Pipeline pipeline) { - return false; - } - - @Override - public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { + public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { for (DatanodeDetails dn : pipeline.getNodes()) { - nodeManager.recordPendingAllocationForDatanode(dn.getID(), containerID); + if (!nodeManager.checkSpaceAndRecordAllocation(nodeManager.getDatanodeInfo(dn), containerID)) { + return false; + } } + return true; } @Override diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index 4e53f6d629ae..7a9e5b4e8196 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -17,7 +17,6 @@ package org.apache.hadoop.hdds.scm.pipeline; -import static org.apache.hadoop.hdds.client.ReplicationFactor.THREE; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_PIPELINE_ALLOCATED_TIMEOUT; @@ -50,7 +49,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.common.collect.ImmutableList; import java.io.File; import java.io.IOException; import java.time.Instant; @@ -66,11 +64,9 @@ import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; -import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeID; -import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; @@ -116,7 +112,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; /** * Tests for PipelineManagerImpl. @@ -935,48 +930,6 @@ public void testCreatePipelineForRead() throws IOException { } } - /** - * {@link PipelineManager#hasEnoughSpace(Pipeline)} should return false if all the - * volumes on any Datanode in the pipeline have space less than or equal to the configured container size. - */ - @Test - public void testHasEnoughSpace() throws IOException { - NodeManager mockedNodeManager = Mockito.mock(NodeManager.class); - PipelineManagerImpl pipelineManager = PipelineManagerImpl.newPipelineManager(conf, - SCMHAManagerStub.getInstance(true), - mockedNodeManager, - SCMDBDefinition.PIPELINES.getTable(dbStore), - new EventQueue(), - scmContext, - serviceManager, - testClock); - - DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails(); - DatanodeDetails dn2 = MockDatanodeDetails.randomDatanodeDetails(); - DatanodeDetails dn3 = MockDatanodeDetails.randomDatanodeDetails(); - Pipeline pipeline = Pipeline.newBuilder() - .setId(PipelineID.randomId()) - .setNodes(ImmutableList.of(dn1, dn2, dn3)) - .setState(OPEN) - .setReplicationConfig(ReplicationConfig.fromTypeAndFactor(ReplicationType.RATIS, THREE)) - .build(); - - // Case 1: All nodes have enough space. - doReturn(true).when(mockedNodeManager).hasSpaceForNewContainerAllocation(dn1.getID()); - doReturn(true).when(mockedNodeManager).hasSpaceForNewContainerAllocation(dn2.getID()); - doReturn(true).when(mockedNodeManager).hasSpaceForNewContainerAllocation(dn3.getID()); - assertTrue(pipelineManager.hasEnoughSpace(pipeline)); - - // Case 2: One node does not have enough space — pipeline should be rejected. - doReturn(false).when(mockedNodeManager).hasSpaceForNewContainerAllocation(dn1.getID()); - assertFalse(pipelineManager.hasEnoughSpace(pipeline)); - - // Case 3: All nodes do not have enough space. - doReturn(false).when(mockedNodeManager).hasSpaceForNewContainerAllocation(dn2.getID()); - doReturn(false).when(mockedNodeManager).hasSpaceForNewContainerAllocation(dn3.getID()); - assertFalse(pipelineManager.hasEnoughSpace(pipeline)); - } - private Set createContainerReplicasList( List dns) { Set replicas = new HashSet<>(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTrackerIntegration.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTrackerIntegration.java new file mode 100644 index 000000000000..13d6bf49fa0e --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTrackerIntegration.java @@ -0,0 +1,184 @@ +/* + * 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.hdds.scm.container; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +import java.util.function.BooleanSupplier; +import org.apache.hadoop.hdds.HddsConfigKeys; +import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.scm.node.PendingContainerTracker; +import org.apache.hadoop.hdds.scm.node.SCMNodeManager; +import org.apache.hadoop.hdds.scm.node.SCMNodeMetrics; +import org.apache.hadoop.hdds.scm.server.StorageContainerManager; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.TestDataUtil; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Integration tests for PendingContainerTracker. + */ +@Timeout(300) +public class TestPendingContainerTrackerIntegration { + + private static final Logger LOG = + LoggerFactory.getLogger(TestPendingContainerTrackerIntegration.class); + private MiniOzoneCluster cluster; + private OzoneClient client; + private ContainerManager containerManager; + private SCMNodeMetrics metrics; + private OzoneBucket bucket; + + @BeforeEach + public void setup() throws Exception { + OzoneConfiguration conf = new OzoneConfiguration(); + + conf.set(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, "60s"); + + // Reduce heartbeat interval for faster container reports + conf.set(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, "10s"); + + conf.set("ozone.scm.container.size", "100MB"); + conf.set("ozone.scm.pipeline.owner.container.count", "1"); + conf.set("ozone.scm.pipeline.per.metadata.disk", "1"); + conf.set("ozone.scm.datanode.pipeline.limit", "1"); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(3) + .build(); + cluster.waitForClusterToBeReady(); + cluster.waitTobeOutOfSafeMode(); + + StorageContainerManager scm = cluster.getStorageContainerManager(); + containerManager = scm.getContainerManager(); + client = cluster.newClient(); + + // Create bucket for testing + bucket = TestDataUtil.createVolumeAndBucket(client); + + SCMNodeManager nodeManager = (SCMNodeManager) scm.getScmNodeManager(); + assertNotNull(nodeManager); + PendingContainerTracker pendingTracker = nodeManager.getPendingContainerTracker(); + assertNotNull(pendingTracker, "PendingContainerTracker should be initialized"); + metrics = pendingTracker.getMetrics(); + + LOG.info("Test setup complete - ICR interval: 5s, Heartbeat interval: 1s"); + } + + @AfterEach + public void cleanup() throws Exception { + if (client != null) { + client.close(); + } + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Test: Write key → Container allocation → Pending tracked → ICR → Pending removed. + */ + @Test + public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { + long initialAdded = metrics.getNumPendingContainersAdded(); + long initialRemoved = metrics.getNumPendingContainersRemoved(); + + // Allocate a container directly + containerManager.allocateContainer( + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + "omServiceIdDefault"); + + // Verify the added metric increased, meaning pending was recorded + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersAdded() > initialAdded, + 100, 5000); + + long afterAdded = metrics.getNumPendingContainersAdded(); + assertThat(afterAdded).isGreaterThan(initialAdded); + + LOG.info("Pending tracked successfully. Waiting for ICR to remove pending..."); + + // Write a key so datanodes send ICRs + String keyName = "testKey1"; + byte[] data = "Testing Pending Container Tracker".getBytes(UTF_8); + + LOG.info("Writing key: {}", keyName); + try (OzoneOutputStream out = bucket.createKey(keyName, data.length, + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + new java.util.HashMap<>())) { + out.write(data); + } + LOG.info("Key written successfully"); + + // Wait for ICRs to be processed and removed metric to increase + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersRemoved() > initialRemoved, + 100, 5000); + + long afterRemoved = metrics.getNumPendingContainersRemoved(); + assertThat(afterRemoved).isGreaterThan(initialRemoved); + + LOG.info("After added={}, removed={}", afterAdded, afterRemoved); + } + + /** + * Test: Verify metrics are updated correctly. + */ + @Test + public void testMetricsUpdateThroughLifecycle() throws Exception { + long initialAdded = metrics.getNumPendingContainersAdded(); + long initialRemoved = metrics.getNumPendingContainersRemoved(); + + LOG.info("Initial metrics: added={}, removed={}", initialAdded, initialRemoved); + + // Write multiple keys + for (int i = 0; i < 3; i++) { + String keyName = "metricsTestKey" + i; + byte[] data = ("Metrics test " + i).getBytes(UTF_8); + + try (OzoneOutputStream out = bucket.createKey(keyName, data.length, + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + new java.util.HashMap<>())) { + out.write(data); + } + } + + // addedMetrics should increase as containers are allocated + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersAdded() > initialAdded, + 100, 5000); + + // Removed metric should increase after ICR processing + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersRemoved() > initialRemoved, + 100, 5000); + } +}