From b1b469e0c6867744241d09afcf4696a9742a1419 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 02:54:02 +0530 Subject: [PATCH 01/22] HDDS-14921. Improve space accounting in SCM with In-Flight container allocation tracking. --- .../org/apache/hadoop/hdds/scm/ScmConfig.java | 21 + .../src/main/resources/ozone-default.xml | 11 + .../scm/container/ContainerManagerImpl.java | 2 + .../scm/container/ContainerReportHandler.java | 9 + .../IncrementalContainerReportHandler.java | 16 + .../container/PendingContainerTracker.java | 338 ++++++++++++++++ .../metrics/SCMContainerManagerMetrics.java | 22 + .../replication/ReplicationManager.java | 38 +- .../TestPendingContainerTracker.java | 375 ++++++++++++++++++ .../scm/pipeline/MockPipelineManager.java | 5 + ...estPendingContainerTrackerIntegration.java | 269 +++++++++++++ 11 files changed, 1105 insertions(+), 1 deletion(-) create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java create mode 100644 hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTrackerIntegration.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java index 9d3d7ef85cbe..1bd8f2986f70 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java @@ -138,10 +138,31 @@ public class ScmConfig extends ReconfigurableConfig { ) private int transactionToDNsCommitMapLimit = 5000000; + @Config(key = "hdds.scm.container.pending-allocation.roll-interval", + defaultValue = "10m", + type = ConfigType.TIME, + tags = { ConfigTag.SCM, ConfigTag.CONTAINER }, + description = + "Time interval for rolling the pending container allocation window. " + + "Pending container allocations are tracked in a two-window tumbling bucket " + + "pattern. Each window has this duration. " + + "After 2x this interval, allocations that haven't been confirmed via " + + "container reports will automatically age out. Default is 10 minutes." + ) + private Duration pendingContainerAllocationRollInterval = Duration.ofMinutes(10); + public int getTransactionToDNsCommitMapLimit() { return transactionToDNsCommitMapLimit; } + public Duration getPendingContainerAllocationRollInterval() { + return pendingContainerAllocationRollInterval; + } + + public void setPendingContainerAllocationRollInterval(Duration duration) { + this.pendingContainerAllocationRollInterval = duration; + } + public Duration getBlockDeletionInterval() { return blockDeletionInterval; } diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 2cb1f52ede15..da641a2708c6 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -379,6 +379,17 @@ datanode periodically send container report to SCM. Unit could be defined with postfix (ns,ms,s,m,h,d) + + hdds.scm.container.pending-allocation.roll-interval + 10m + OZONE, CONTAINER, MANAGEMENT + Time interval for rolling the pending container allocation window. + Pending container allocations are tracked in a two-window tumbling bucket + pattern. Each window has this duration. + After 2x this interval, allocations that haven't been confirmed via + container reports will automatically age out. Default is 10 minutes. + + hdds.pipeline.report.interval 60000ms 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 15a566f6421b..85ce9f67a844 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 @@ -270,6 +270,8 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, containerStateManager.addContainer(containerInfoBuilder.build()); scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); + // Record pending allocation - tracks containers scheduled but not yet written + pipelineManager.recordPendingAllocation(pipeline, containerID); return containerStateManager.getContainer(containerID); } 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..b9358096d5d7 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 @@ -175,6 +175,15 @@ 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 + // No need to remove on subsequent reports (it's already been removed) + if (container != null && getContainerManager() instanceof ContainerManagerImpl) { + ((ContainerManagerImpl) getContainerManager()) + .getPendingContainerTracker() + .removePendingAllocation(datanodeDetails, 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..ff4eae77f97d 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 @@ -89,6 +89,15 @@ protected void processICR(IncrementalContainerReportFromDatanode report, ContainerID id = ContainerID.valueOf(replicaProto.getContainerID()); final ContainerInfo container; try { + // Check if container is already known to this DN before adding + boolean alreadyOnDn = false; + try { + alreadyOnDn = getNodeManager().getContainers(dd).contains(id); + } catch (NodeNotFoundException e) { + // DN not found, treat as not already on DN + getLogger().debug("Datanode not found when checking containers: {}", dd); + } + try { container = getContainerManager().getContainer(id); // Ensure we reuse the same ContainerID instance in containerInfo @@ -103,6 +112,13 @@ protected void processICR(IncrementalContainerReportFromDatanode report, } if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); + + // Remove from pending tracker when container is added to DN + if (!alreadyOnDn && getContainerManager() instanceof ContainerManagerImpl) { + ((ContainerManagerImpl) getContainerManager()) + .getPendingContainerTracker() + .removePendingAllocation(dd, id); + } } success = true; } catch (ContainerNotFoundException e) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java new file mode 100644 index 000000000000..083200ec220b --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java @@ -0,0 +1,338 @@ +/* + * 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 com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.scm.container.metrics.SCMContainerManagerMetrics; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.util.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Tracks pending container allocations using a Two Window Tumbling Bucket pattern. + * Similar like HDFS HADOOP-3707. + * + * Two Window Tumbling Bucket for automatic aging and cleanup. + * + * How It Works: + *
  • Each DataNode has two sets: currentWindow and previousWindow
  • + *
  • New allocations go into currentWindow
  • + *
  • Every ROLL_INTERVAL (default 10 minutes): + *
      + *
    • previousWindow = currentWindow (shift)
    • + *
    • currentWindow = new empty set (reset)
    • + *
    • Old previousWindow is discarded (automatic aging)
    • + *
    + *
  • + *
  • When checking pending: return union of currentWindow + previousWindow
  • + * + * + * Example Timeline: + *
    + * Time  | Action                    | CurrentWindow | PreviousWindow | Total Pending
    + * ------+---------------------------+---------------+----------------+--------------
    + * 00:00 | Allocate Container-1      | {C1}          | {}             | {C1}
    + * 00:05 | Allocate Container-2      | {C1, C2}      | {}             | {C1, C2}
    + * 00:10 | [ROLL] Window tumbles     | {}            | {C1, C2}       | {C1, C2}
    + * 00:12 | Allocate Container-3      | {C3}          | {C1, C2}       | {C1, C2, C3}
    + * 00:15 | Report confirms C1        | {C3}          | {C2}           | {C2, C3}
    + * 00:20 | [ROLL] Window tumbles     | {}            | {C3}           | {C3}
    + *       | (C2 aged out if not reported)
    + * 
    + * + */ +public class PendingContainerTracker { + + private static final Logger LOG = LoggerFactory.getLogger(PendingContainerTracker.class); + + /** + * Roll interval in milliseconds. + * Configurable via hdds.scm.container.pending-allocation.roll-interval. + * Default: 10 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set currentWindow = ConcurrentHashMap.newKeySet(); + private Set previousWindow = ConcurrentHashMap.newKeySet(); + private long lastRollTime = Time.monotonicNow(); + private final long rollIntervalMs; + + TwoWindowBucket(long rollIntervalMs) { + this.rollIntervalMs = rollIntervalMs; + } + + /** + * Roll the windows: previous = current, current = empty. + * Called when current time exceeds lastRollTime + rollIntervalMs. + */ + synchronized void rollIfNeeded() { + long now = Time.monotonicNow(); + if (now - lastRollTime >= rollIntervalMs) { + // Shift: current becomes previous + previousWindow = currentWindow; + // Reset: new empty current window + currentWindow = ConcurrentHashMap.newKeySet(); + lastRollTime = now; + LOG.debug("Rolled window. Previous window size: {}, Current window reset to empty", previousWindow.size()); + } + } + + /** + * Get union of both windows (all pending containers). + */ + synchronized Set getAllPending() { + Set all = new HashSet<>(); + all.addAll(currentWindow); + all.addAll(previousWindow); + return all; + } + + /** + * Add container to current window. + */ + synchronized boolean add(ContainerID containerID) { + return currentWindow.add(containerID); + } + + /** + * Remove container from both windows. + */ + synchronized boolean remove(ContainerID containerID) { + boolean removedFromCurrent = currentWindow.remove(containerID); + boolean removedFromPrevious = previousWindow.remove(containerID); + return removedFromCurrent || removedFromPrevious; + } + + /** + * Check if either window is non-empty. + */ + synchronized boolean isEmpty() { + return currentWindow.isEmpty() && previousWindow.isEmpty(); + } + + /** + * Get count of all pending containers (union). + */ + synchronized int getCount() { + return getAllPending().size(); + } + } + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers. + */ + private final SCMContainerManagerMetrics metrics; + + public PendingContainerTracker(long maxContainerSize) { + this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes + } + + public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, + SCMContainerManagerMetrics metrics) { + this.datanodeBuckets = new ConcurrentHashMap<>(); + this.maxContainerSize = maxContainerSize; + this.rollIntervalMs = rollIntervalMs; + this.metrics = metrics; + LOG.info("PendingContainerTracker initialized with maxContainerSize={}B, rollInterval={}ms", + maxContainerSize, rollIntervalMs); + } + + /** + * Record a pending container allocation for all DataNodes in the pipeline. + * Container is added to the current window. + * + * @param pipeline The pipeline where container is allocated + * @param containerID The container being allocated + */ + public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { + if (pipeline == null || containerID == null) { + LOG.warn("Ignoring null pipeline or containerID"); + return; + } + + for (DatanodeDetails node : pipeline.getNodes()) { + recordPendingAllocationForDatanode(node, containerID); + } + } + + /** + * Record a pending container allocation for a single DataNode. + * Container is added to the current window. + * + * @param node The DataNode where container is being allocated/replicated + * @param containerID The container being allocated/replicated + */ + public void recordPendingAllocationForDatanode(DatanodeDetails node, ContainerID containerID) { + if (node == null || containerID == null) { + LOG.warn("Ignoring null node or containerID"); + return; + } + + TwoWindowBucket bucket = datanodeBuckets.computeIfAbsent( + node.getUuid(), + k -> new TwoWindowBucket(rollIntervalMs) + ); + + // Roll window if needed before adding + bucket.rollIfNeeded(); + + boolean added = bucket.add(containerID); + LOG.info("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", + containerID, node.getUuidString(), added, bucket.getCount()); + + // Increment metrics counter + if (added && metrics != null) { + metrics.incNumPendingContainersAdded(); + } + } + + /** + * Remove a pending container allocation from a specific DataNode. + * Removes from both current and previous windows. + * Called when container is confirmed. + * + * @param node The DataNode + * @param containerID The container to remove from pending + */ + public void removePendingAllocation(DatanodeDetails node, ContainerID containerID) { + if (node == null || containerID == null) { + return; + } + + TwoWindowBucket bucket = datanodeBuckets.get(node.getUuid()); + if (bucket != null) { + // Roll window if needed before removing + bucket.rollIfNeeded(); + + boolean removed = bucket.remove(containerID); + LOG.info("Removed pending container {} from DataNode {}. Removed={}, Remaining={}", + containerID, node.getUuidString(), removed, bucket.getCount()); + + // Increment metrics counter + if (removed && metrics != null) { + metrics.incNumPendingContainersRemoved(); + } + + // Cleanup empty buckets to prevent memory leak + if (bucket.isEmpty()) { + LOG.info("Cleanup pending bucket"); + datanodeBuckets.remove(node.getUuid(), bucket); + } + } + } + + /** + * Get the total size of pending allocations on a DataNode. + * Returns union of current and previous windows. + * + * @param node The DataNode + * @return Total bytes of pending container allocations + */ + public long getPendingAllocationSize(DatanodeDetails node) { + if (node == null) { + return 0; + } + + TwoWindowBucket bucket = datanodeBuckets.get(node.getUuid()); + LOG.info("Get pending from DataNode {}", + node.getUuidString()); + if (bucket == null) { + LOG.info("Get pending from DataNode {} is null", + node.getUuidString()); + return 0; + } + + // Roll window if needed before querying + bucket.rollIfNeeded(); + + // Each pending container assumes max size + return (long) bucket.getCount() * maxContainerSize; + } + + /** + * Get the set of pending container IDs for a DataNode. + * Returns union of current and previous windows. + * Useful for debugging and monitoring. + * + * @param node The DataNode + * @return Set of pending container IDs + */ + public Set getPendingContainers(DatanodeDetails node) { + if (node == null) { + return Collections.emptySet(); + } + + TwoWindowBucket bucket = datanodeBuckets.get(node.getUuid()); + if (bucket == null) { + return Collections.emptySet(); + } + + bucket.rollIfNeeded(); + return bucket.getAllPending(); + } + + /** + * Get total number of DataNodes with pending allocations. + * + * @return Count of DataNodes + */ + public int getDataNodeCount() { + return datanodeBuckets.size(); + } + + /** + * Get total number of pending containers across all DataNodes. + * Note: Same container on multiple DataNodes is counted once per DataNode. + * The count may include containers from the previous window (up to 10 minutes old). + * + * @return Total pending container count + */ + public long getTotalPendingCount() { + return datanodeBuckets.values().stream() + .mapToLong(TwoWindowBucket::getCount) + .sum(); + } + + @VisibleForTesting + public SCMContainerManagerMetrics getMetrics() { + return metrics; + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java index ed3670ea6645..32993984285d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java @@ -48,6 +48,10 @@ public final class SCMContainerManagerMetrics { private @Metric MutableCounterLong numICRReportsProcessedSuccessful; private @Metric MutableCounterLong numICRReportsProcessedFailed; + // Pending container allocation metrics + private @Metric MutableCounterLong numPendingContainersAdded; + private @Metric MutableCounterLong numPendingContainersRemoved; + private SCMContainerManagerMetrics() { } @@ -141,4 +145,22 @@ public long getNumListContainersOps() { return numListContainerOps.value(); } + // Pending container allocation metrics + + public void incNumPendingContainersAdded() { + this.numPendingContainersAdded.incr(); + } + + public long getNumPendingContainersAdded() { + return numPendingContainersAdded.value(); + } + + public void incNumPendingContainersRemoved() { + this.numPendingContainersRemoved.incr(); + } + + public long getNumPendingContainersRemoved() { + return numPendingContainersRemoved.value(); + } + } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java index 8cd8444d1d2f..cecb942e2e99 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java @@ -58,8 +58,10 @@ 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; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import org.apache.hadoop.hdds.scm.container.PendingContainerTracker; import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; import org.apache.hadoop.hdds.scm.container.replication.health.ClosedWithUnhealthyReplicasHandler; import org.apache.hadoop.hdds.scm.container.replication.health.ClosingContainerHandler; @@ -675,6 +677,30 @@ public void sendDatanodeCommand(SCMCommand command, scmDeadlineEpochMs); } + /** + * Record a pending container replication in the PendingContainerTracker. + * This prevents target DNs from being selected for more work when they + * already have pending replications that will consume space. + * + * @param containerID The container being replicated + * @param target The target DataNode that will receive the replica + */ + private void recordPendingReplication(ContainerID containerID, DatanodeDetails target) { + if (containerManager instanceof ContainerManagerImpl) { + try { + PendingContainerTracker tracker = + ((ContainerManagerImpl) containerManager).getPendingContainerTracker(); + tracker.recordPendingAllocationForDatanode(target, containerID); + + LOG.debug("Recorded pending replication of container {} to DataNode {}", + containerID, target.getUuidString()); + } catch (Exception e) { + LOG.warn("Failed to record pending replication of container {} to DataNode {}", + containerID, target.getUuidString(), e); + } + } + } + private void adjustPendingOpsAndMetrics(ContainerInfo containerInfo, SCMCommand cmd, DatanodeDetails targetDatanode, long scmDeadlineEpochMs) { @@ -694,20 +720,26 @@ private void adjustPendingOpsAndMetrics(ContainerInfo containerInfo, final ByteString targetIndexes = rcc.getMissingContainerIndexes(); long requiredSize = HddsServerUtil.requiredReplicationSpace(containerInfo.getUsedBytes()); for (int i = 0; i < targetIndexes.size(); i++) { - containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), targets.get(i), + DatanodeDetails target = targets.get(i); + containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), target, targetIndexes.byteAt(i), cmd, scmDeadlineEpochMs, requiredSize, clock.millis()); + + // Track EC reconstruction in PendingContainerTracker + recordPendingReplication(containerInfo.containerID(), target); } getMetrics().incrEcReconstructionCmdsSentTotal(); } else if (cmd.getType() == Type.replicateContainerCommand) { ReplicateContainerCommand rcc = (ReplicateContainerCommand) cmd; long requiredSize = HddsServerUtil.requiredReplicationSpace(containerInfo.getUsedBytes()); + DatanodeDetails replicationTarget; if (rcc.getTargetDatanode() == null) { /* This means the target will pull a replica from a source, so the op's target Datanode should be the Datanode this command is being sent to. */ + replicationTarget = targetDatanode; containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), targetDatanode, rcc.getReplicaIndex(), cmd, scmDeadlineEpochMs, requiredSize, clock.millis()); } else { @@ -715,10 +747,14 @@ private void adjustPendingOpsAndMetrics(ContainerInfo containerInfo, This means the source will push replica to the target, so the op's target Datanode should be the Datanode the replica will be pushed to. */ + replicationTarget = rcc.getTargetDatanode(); containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), rcc.getTargetDatanode(), rcc.getReplicaIndex(), cmd, scmDeadlineEpochMs, requiredSize, clock.millis()); } + // Track replication in PendingContainerTracker to prevent space exhaustion + recordPendingReplication(containerInfo.containerID(), replicationTarget); + if (rcc.getReplicaIndex() > 0) { getMetrics().incrEcReplicationCmdsSentTotal(); } else if (rcc.getReplicaIndex() == 0) { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java new file mode 100644 index 000000000000..f7126cf678ba --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java @@ -0,0 +1,375 @@ +/* + * 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 org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +import java.io.IOException; +import java.util.Set; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * Tests for PendingContainerTracker. + */ +public class TestPendingContainerTracker { + + private static final long MAX_CONTAINER_SIZE = 5L * 1024 * 1024 * 1024; // 5GB + + private PendingContainerTracker tracker; + private Pipeline pipeline; + private DatanodeDetails dn1; + private DatanodeDetails dn2; + private DatanodeDetails dn3; + private ContainerID container1; + private ContainerID container2; + private ContainerID container3; + + @BeforeEach + public void setUp() throws IOException { + tracker = new PendingContainerTracker(MAX_CONTAINER_SIZE); + + // Create a 3-node Ratis pipeline + pipeline = MockPipeline.createPipeline(3); + dn1 = pipeline.getNodes().get(0); + dn2 = pipeline.getNodes().get(1); + dn3 = pipeline.getNodes().get(2); + + container1 = ContainerID.valueOf(1L); + container2 = ContainerID.valueOf(2L); + container3 = ContainerID.valueOf(3L); + } + + @Test + public void testRecordPendingAllocation() { + // Initially no pending containers + assertEquals(0, tracker.getPendingContainers(dn1).size()); + assertEquals(0, tracker.getPendingAllocationSize(dn1)); + + // Record a pending allocation + tracker.recordPendingAllocation(pipeline, container1); + + // All 3 DNs should have the container pending + assertEquals(1, tracker.getPendingContainers(dn1).size()); + assertEquals(1, tracker.getPendingContainers(dn2).size()); + assertEquals(1, tracker.getPendingContainers(dn3).size()); + + // Size should be MAX_CONTAINER_SIZE for each DN + assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); + assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn2)); + assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn3)); + } + + @Test + public void testRecordMultiplePendingAllocations() { + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container2); + tracker.recordPendingAllocation(pipeline, container3); + + // Each DN should have 3 pending containers + assertEquals(3, tracker.getPendingContainers(dn1).size()); + assertEquals(3, tracker.getPendingContainers(dn2).size()); + assertEquals(3, tracker.getPendingContainers(dn3).size()); + + // Size should be 3 × MAX_CONTAINER_SIZE + assertEquals(3 * MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); + } + + @Test + public void testIdempotentRecording() { + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container1); // Duplicate + + // Should still be 1 container (Set deduplication) + assertEquals(1, tracker.getPendingContainers(dn1).size()); + assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); + } + + @Test + public void testRemovePendingAllocation() { + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container2); + + assertEquals(2, tracker.getPendingContainers(dn1).size()); + + // Remove one container from DN1 + tracker.removePendingAllocation(dn1, container1); + + assertEquals(1, tracker.getPendingContainers(dn1).size()); + assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); + + // DN2 and DN3 should still have both containers + assertEquals(2, tracker.getPendingContainers(dn2).size()); + assertEquals(2, tracker.getPendingContainers(dn3).size()); + } + + @Test + public void testRemovePendingAllocationFromPipeline() { + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container2); + + // Remove container1 from all nodes in pipeline + for (DatanodeDetails dn : pipeline.getNodes()) { + tracker.removePendingAllocation(dn, container1); + } + + // All DNs should have only container2 remaining + assertEquals(1, tracker.getPendingContainers(dn1).size()); + assertEquals(1, tracker.getPendingContainers(dn2).size()); + assertEquals(1, tracker.getPendingContainers(dn3).size()); + } + + @Test + public void testRemoveNonExistentContainer() { + tracker.recordPendingAllocation(pipeline, container1); + + // Remove a container that was never added - should not throw exception + tracker.removePendingAllocation(dn1, container2); + + // DN1 should still have container1 + assertEquals(1, tracker.getPendingContainers(dn1).size()); + } + + @Test + public void testGetPendingContainers() { + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container2); + + Set pending = tracker.getPendingContainers(dn1); + + assertEquals(2, pending.size()); + assertThat(pending.contains(container1)); + assertThat(pending.contains(container2)); + + // Returned set should be a copy - modifying it shouldn't affect tracker + pending.add(container3); + assertEquals(2, tracker.getPendingContainers(dn1).size()); // Should still be 2 + } + + @Test + public void testGetPendingContainersForNonExistentDN() { + DatanodeDetails unknownDN = MockDatanodeDetails.randomDatanodeDetails(); + + Set pending = tracker.getPendingContainers(unknownDN); + + assertThat(pending.isEmpty()); + } + + @Test + public void testGetTotalPendingCount() { + assertEquals(0, tracker.getTotalPendingCount()); + + tracker.recordPendingAllocation(pipeline, container1); + + // 1 container × 3 DNs = 3 total pending + assertEquals(3, tracker.getTotalPendingCount()); + + tracker.recordPendingAllocation(pipeline, container2); + + // 2 containers × 3 DNs = 6 total pending + assertEquals(6, tracker.getTotalPendingCount()); + + // Remove from one DN + tracker.removePendingAllocation(dn1, container1); + + // (2 containers × 2 DNs) + (1 container × 1 DN) = 5 total + assertEquals(5, tracker.getTotalPendingCount()); + } + + @Test + public void testConcurrentModification() throws InterruptedException { + // Test thread-safety by having multiple threads add/remove containers + final int numThreads = 10; + final int operationsPerThread = 100; + + Thread[] threads = new Thread[numThreads]; + + for (int i = 0; i < numThreads; i++) { + final int threadId = i; + threads[i] = new Thread(() -> { + for (int j = 0; j < operationsPerThread; j++) { + ContainerID cid = ContainerID.valueOf(threadId * 1000L + j); + tracker.recordPendingAllocation(pipeline, cid); + + if (j % 2 == 0) { + tracker.removePendingAllocation(dn1, cid); + } + } + }); + } + + // Start all threads + for (Thread thread : threads) { + thread.start(); + } + + // Wait for all to finish + for (Thread thread : threads) { + thread.join(); + } + + // Verify no exceptions occurred and counts are reasonable + assertThat(tracker.getTotalPendingCount() >= 0); + assertThat(tracker.getDataNodeCount() <= 3); + } + + @Test + public void testMemoryCleanupOnEmptySet() { + tracker.recordPendingAllocation(pipeline, container1); + + assertEquals(3, tracker.getDataNodeCount()); + + // Remove the only pending container from DN1 + tracker.removePendingAllocation(dn1, container1); + + // DN1 should be removed from the map (memory cleanup) + assertEquals(2, tracker.getDataNodeCount()); + } + + @Test + public void testPendingContainer() { + // Simulate allocation and confirmation flow + + // Allocate 3 containers + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container2); + tracker.recordPendingAllocation(pipeline, container3); + + // Each DN should have 3 pending, 15GB total + assertEquals(3, tracker.getPendingContainers(dn1).size()); + assertEquals(3 * MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); + + // DN1 confirms container1 via container report + tracker.removePendingAllocation(dn1, container1); + + // DN1 now has 2 pending, 10GB + assertEquals(2, tracker.getPendingContainers(dn1).size()); + assertEquals(2 * MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); + + // DN2 and DN3 still have 3 pending + assertEquals(3, tracker.getPendingContainers(dn2).size()); + assertEquals(3, tracker.getPendingContainers(dn3).size()); + + // All DNs eventually confirm all containers + for (DatanodeDetails dn : pipeline.getNodes()) { + tracker.removePendingAllocation(dn, container1); + tracker.removePendingAllocation(dn, container2); + tracker.removePendingAllocation(dn, container3); + } + + // All DNs should have 0 pending + assertEquals(0, tracker.getPendingContainers(dn1).size()); + assertEquals(0, tracker.getPendingContainers(dn2).size()); + assertEquals(0, tracker.getPendingContainers(dn3).size()); + assertEquals(0, tracker.getTotalPendingCount()); + assertEquals(0, tracker.getDataNodeCount()); + } + + @Test + public void testRemoveFromBothWindows() { + // This test verifies that removal works from both current and previous windows + // In general, a container could be in previous window after a roll + + // Add containers + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container2); + + assertEquals(2, tracker.getPendingContainers(dn1).size()); + + // Remove container1 - should work regardless of which window it's in + tracker.removePendingAllocation(dn1, container1); + + assertEquals(1, tracker.getPendingContainers(dn1).size()); + + Set pending = tracker.getPendingContainers(dn1); + assertFalse(pending.contains(container1)); + assertThat(pending.contains(container2)); + } + + @Test + public void testUnionOfBothWindows() { + // This test verifies the two-window concept: + // getPendingContainers should return union of current + previous windows + + // Add container1 + tracker.recordPendingAllocation(pipeline, container1); + + assertEquals(1, tracker.getPendingContainers(dn1).size()); + Set pending1 = tracker.getPendingContainers(dn1); + assertThat(pending1.contains(container1)); + + // Add container2 - should be in same window initially + tracker.recordPendingAllocation(pipeline, container2); + + assertEquals(2, tracker.getPendingContainers(dn1).size()); + Set pending2 = tracker.getPendingContainers(dn1); + assertThat(pending2.contains(container1)); + assertThat(pending2.contains(container2)); + + // Both containers should be in the union + assertEquals(2, pending2.size()); + } + + @Test + public void testIdempotencyAcrossWindows() { + // Recording same container multiple times should only count it once + // This should work even if it spans windows + + tracker.recordPendingAllocation(pipeline, container1); + assertEquals(1, tracker.getPendingContainers(dn1).size()); + + // Record again - should still be 1 (idempotency via Set) + tracker.recordPendingAllocation(pipeline, container1); + assertEquals(1, tracker.getPendingContainers(dn1).size()); + + // Add different container + tracker.recordPendingAllocation(pipeline, container2); + assertEquals(2, tracker.getPendingContainers(dn1).size()); + + // Record container1 again + tracker.recordPendingAllocation(pipeline, container1); + assertEquals(2, tracker.getPendingContainers(dn1).size()); // Still 2, not 3 + } + + @Test + public void testExplicitRemoval() { + + tracker.recordPendingAllocation(pipeline, container1); + tracker.recordPendingAllocation(pipeline, container2); + tracker.recordPendingAllocation(pipeline, container3); + + assertEquals(3, tracker.getPendingContainers(dn1).size()); + + // Simulate container report confirms container1 and container2 + tracker.removePendingAllocation(dn1, container1); + tracker.removePendingAllocation(dn1, container2); + + // Immediately reflects the removal (doesn't wait for aging) + assertEquals(1, tracker.getPendingContainers(dn1).size()); + + Set pending = tracker.getPendingContainers(dn1); + assertEquals(1, pending.size()); + assertThat(pending.contains(container3)); + } +} 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..abcb916a16ac 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 @@ -355,4 +355,9 @@ public int openContainerLimit(List datanodes) { public SCMPipelineMetrics getMetrics() { return null; } + + @Override + public org.apache.hadoop.hdds.scm.node.DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails) { + return null; + } } 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..d301c19e2d9e --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTrackerIntegration.java @@ -0,0 +1,269 @@ +/* + * 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.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +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.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.container.metrics.SCMContainerManagerMetrics; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +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 OzoneConfiguration conf; + private MiniOzoneCluster cluster; + private StorageContainerManager scm; + private OzoneClient client; + private ContainerManager containerManager; + private PendingContainerTracker pendingTracker; + private SCMContainerManagerMetrics metrics; + private OzoneBucket bucket; + + @BeforeEach + public void setup() throws Exception { + 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(); + + scm = cluster.getStorageContainerManager(); + containerManager = scm.getContainerManager(); + client = cluster.newClient(); + + // Create bucket for testing + bucket = TestDataUtil.createVolumeAndBucket(client); + + // Get the pending tracker + if (containerManager instanceof ContainerManagerImpl) { + pendingTracker = ((ContainerManagerImpl) containerManager) + .getPendingContainerTracker(); + assertNotNull(pendingTracker, "PendingContainerTracker should be initialized"); + } + metrics = pendingTracker.getMetrics(); + // metrics = SCMContainerManagerMetrics.create(); + + LOG.info("Test setup complete - ICR interval: 5s, Heartbeat interval: 1s"); + } + + @AfterEach + public void cleanup() throws Exception { + if (metrics != null) { + metrics.unRegister(); + } + 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 + ContainerInfo container = containerManager.allocateContainer( + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + "omServiceIdDefault"); + + // Find the container that was allocated + ContainerInfo containerInfo = scm.getContainerManager().getContainers().get(0); + ContainerWithPipeline containerWithPipeline = + scm.getClientProtocolServer().getContainerWithPipeline( + containerInfo.getContainerID()); + + Pipeline pipeline = containerWithPipeline.getPipeline(); + + // Verify pending containers are tracked for all nodes in pipeline + List nodesWithPending = new ArrayList<>(); + for (DatanodeDetails dn : pipeline.getNodes()) { + long pendingSize = pendingTracker.getPendingAllocationSize(dn); + if (pendingSize > 0) { + nodesWithPending.add(dn); + LOG.info("DataNode {} has {} bytes pending", dn.getUuidString(), pendingSize); + + Set pendingContainers = pendingTracker.getPendingContainers(dn); + assertThat(pendingContainers.contains(container.containerID())); + } + } + + assertThat(nodesWithPending.size() > 0); + + // Verify metrics increased + long afterAdded = metrics.getNumPendingContainersAdded(); + assertThat(afterAdded > initialAdded); + + LOG.info("Pending tracked successfully. Waiting for ICR to remove pending..."); + + // Write a key + String keyName = "testKey1"; + byte[] data = "Hello Ozone - 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 sent + GenericTestUtils.waitFor(() -> { + for (DatanodeDetails dn : nodesWithPending) { + Set pendingContainers = pendingTracker.getPendingContainers(dn); + if (pendingContainers.contains(container.containerID())) { + LOG.info("Still waiting for ICR from DN {}", dn.getUuidString()); + return false; + } + } + + LOG.info("All pending containers removed via ICR!"); + return true; + }, 100, 5000); + + // Verify all pending containers removed + for (DatanodeDetails dn : nodesWithPending) { + Set pendingContainers = pendingTracker.getPendingContainers(dn); + assertThat(!pendingContainers.contains(container.containerID())); + } + + // Verify remove metrics increased + long afterRemoved = metrics.getNumPendingContainersRemoved(); + assertThat(afterRemoved > initialRemoved); + + LOG.info("After added = " + afterAdded); + + } + + /** + * Test: Verify idempotency - container reported multiple times. + */ + @Test + public void testIdempotentPendingTracking() throws Exception { + // Allocate a container directly + ContainerInfo container = containerManager.allocateContainer( + RatisReplicationConfig.getInstance(ReplicationFactor.THREE), + "omServiceIdDefault"); + + Pipeline pipeline = scm.getPipelineManager().getPipeline(container.getPipelineID()); + DatanodeDetails firstNode = pipeline.getFirstNode(); + + // Record initial state + long initialSize = pendingTracker.getPendingAllocationSize(firstNode); + int initialCount = pendingTracker.getPendingContainers(firstNode).size(); + + LOG.info("Initial pending state: size={}, count={}", initialSize, initialCount); + + // Try adding the same container again (simulates retry or duplicate allocation) + pendingTracker.recordPendingAllocationForDatanode(firstNode, container.containerID()); + + long afterSize = pendingTracker.getPendingAllocationSize(firstNode); + int afterCount = pendingTracker.getPendingContainers(firstNode).size(); + + // Size and count should remain the same (idempotent) + assertEquals(initialSize, afterSize, + "Pending size should not change when adding duplicate container"); + assertEquals(initialCount, afterCount, + "Pending count should not change when adding duplicate container"); + + } + + /** + * 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(() -> { + long afterAdded = metrics.getNumPendingContainersAdded(); + return afterAdded > initialAdded; + }, 100, 5000); + + // Removed metric should increase after icr process + GenericTestUtils.waitFor(() -> { + long afterRemoved = metrics.getNumPendingContainersRemoved(); + return initialRemoved < afterRemoved; + }, 100, 5000); + } +} From 0e72835f931e6ac3a42415d38b043528891fc778 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 03:06:30 +0530 Subject: [PATCH 02/22] Fix PMD --- .../container/PendingContainerTracker.java | 30 +++++++++---------- ...estPendingContainerTrackerIntegration.java | 6 ++-- 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java index 083200ec220b..4866416b392d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java @@ -74,6 +74,21 @@ public class PendingContainerTracker { * Containers automatically age out after 2 × rollIntervalMs. */ private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers. + */ + private final SCMContainerManagerMetrics metrics; /** * Two-window bucket for a single DataNode. @@ -146,21 +161,6 @@ synchronized int getCount() { } } - /** - * Map of DataNode UUID to TwoWindowBucket. - */ - private final ConcurrentHashMap datanodeBuckets; - - /** - * Maximum container size in bytes. - */ - private final long maxContainerSize; - - /** - * Metrics for tracking pending containers. - */ - private final SCMContainerManagerMetrics metrics; - public PendingContainerTracker(long maxContainerSize) { this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes } 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 index d301c19e2d9e..f2a1280a64b1 100644 --- 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 @@ -55,8 +55,6 @@ public class TestPendingContainerTrackerIntegration { private static final Logger LOG = LoggerFactory.getLogger(TestPendingContainerTrackerIntegration.class); - - private OzoneConfiguration conf; private MiniOzoneCluster cluster; private StorageContainerManager scm; private OzoneClient client; @@ -67,7 +65,7 @@ public class TestPendingContainerTrackerIntegration { @BeforeEach public void setup() throws Exception { - conf = new OzoneConfiguration(); + OzoneConfiguration conf = new OzoneConfiguration(); conf.set(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, "60s"); @@ -151,7 +149,7 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { } } - assertThat(nodesWithPending.size() > 0); + assertThat(!nodesWithPending.isEmpty()); // Verify metrics increased long afterAdded = metrics.getNumPendingContainersAdded(); From bef6fc417d37757d67c8baa24e2db7937577f68c Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 09:53:12 +0530 Subject: [PATCH 03/22] Fix test case --- .../src/main/resources/ozone-default.xml | 11 ------ .../replication/ReplicationManager.java | 38 +------------------ 2 files changed, 1 insertion(+), 48 deletions(-) diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index da641a2708c6..2cb1f52ede15 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -379,17 +379,6 @@ datanode periodically send container report to SCM. Unit could be defined with postfix (ns,ms,s,m,h,d)
    - - hdds.scm.container.pending-allocation.roll-interval - 10m - OZONE, CONTAINER, MANAGEMENT - Time interval for rolling the pending container allocation window. - Pending container allocations are tracked in a two-window tumbling bucket - pattern. Each window has this duration. - After 2x this interval, allocations that haven't been confirmed via - container reports will automatically age out. Default is 10 minutes. - - hdds.pipeline.report.interval 60000ms diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java index cecb942e2e99..8cd8444d1d2f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java @@ -58,10 +58,8 @@ 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; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.PendingContainerTracker; import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; import org.apache.hadoop.hdds.scm.container.replication.health.ClosedWithUnhealthyReplicasHandler; import org.apache.hadoop.hdds.scm.container.replication.health.ClosingContainerHandler; @@ -677,30 +675,6 @@ public void sendDatanodeCommand(SCMCommand command, scmDeadlineEpochMs); } - /** - * Record a pending container replication in the PendingContainerTracker. - * This prevents target DNs from being selected for more work when they - * already have pending replications that will consume space. - * - * @param containerID The container being replicated - * @param target The target DataNode that will receive the replica - */ - private void recordPendingReplication(ContainerID containerID, DatanodeDetails target) { - if (containerManager instanceof ContainerManagerImpl) { - try { - PendingContainerTracker tracker = - ((ContainerManagerImpl) containerManager).getPendingContainerTracker(); - tracker.recordPendingAllocationForDatanode(target, containerID); - - LOG.debug("Recorded pending replication of container {} to DataNode {}", - containerID, target.getUuidString()); - } catch (Exception e) { - LOG.warn("Failed to record pending replication of container {} to DataNode {}", - containerID, target.getUuidString(), e); - } - } - } - private void adjustPendingOpsAndMetrics(ContainerInfo containerInfo, SCMCommand cmd, DatanodeDetails targetDatanode, long scmDeadlineEpochMs) { @@ -720,26 +694,20 @@ private void adjustPendingOpsAndMetrics(ContainerInfo containerInfo, final ByteString targetIndexes = rcc.getMissingContainerIndexes(); long requiredSize = HddsServerUtil.requiredReplicationSpace(containerInfo.getUsedBytes()); for (int i = 0; i < targetIndexes.size(); i++) { - DatanodeDetails target = targets.get(i); - containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), target, + containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), targets.get(i), targetIndexes.byteAt(i), cmd, scmDeadlineEpochMs, requiredSize, clock.millis()); - - // Track EC reconstruction in PendingContainerTracker - recordPendingReplication(containerInfo.containerID(), target); } getMetrics().incrEcReconstructionCmdsSentTotal(); } else if (cmd.getType() == Type.replicateContainerCommand) { ReplicateContainerCommand rcc = (ReplicateContainerCommand) cmd; long requiredSize = HddsServerUtil.requiredReplicationSpace(containerInfo.getUsedBytes()); - DatanodeDetails replicationTarget; if (rcc.getTargetDatanode() == null) { /* This means the target will pull a replica from a source, so the op's target Datanode should be the Datanode this command is being sent to. */ - replicationTarget = targetDatanode; containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), targetDatanode, rcc.getReplicaIndex(), cmd, scmDeadlineEpochMs, requiredSize, clock.millis()); } else { @@ -747,14 +715,10 @@ private void adjustPendingOpsAndMetrics(ContainerInfo containerInfo, This means the source will push replica to the target, so the op's target Datanode should be the Datanode the replica will be pushed to. */ - replicationTarget = rcc.getTargetDatanode(); containerReplicaPendingOps.scheduleAddReplica(containerInfo.containerID(), rcc.getTargetDatanode(), rcc.getReplicaIndex(), cmd, scmDeadlineEpochMs, requiredSize, clock.millis()); } - // Track replication in PendingContainerTracker to prevent space exhaustion - recordPendingReplication(containerInfo.containerID(), replicationTarget); - if (rcc.getReplicaIndex() > 0) { getMetrics().incrEcReplicationCmdsSentTotal(); } else if (rcc.getReplicaIndex() == 0) { From 9f2f85c98da90553122f07b316ec36be5e9596eb Mon Sep 17 00:00:00 2001 From: ashishk Date: Sun, 5 Apr 2026 01:36:48 +0530 Subject: [PATCH 04/22] Move PendingContainerTracker to node package and fix review comments --- .../org/apache/hadoop/hdds/scm/ScmConfig.java | 6 +- .../scm/container/ContainerManagerImpl.java | 25 +- .../scm/container/ContainerReportHandler.java | 11 +- .../IncrementalContainerReportHandler.java | 21 +- .../container/PendingContainerTracker.java | 338 ---------------- .../metrics/SCMContainerManagerMetrics.java | 22 - .../hadoop/hdds/scm/node/DeadNodeHandler.java | 4 + .../hadoop/hdds/scm/node/NodeManager.java | 11 + .../hadoop/hdds/scm/node/SCMNodeManager.java | 48 +++ .../hadoop/hdds/scm/node/SCMNodeMetrics.java | 1 + .../hdds/scm/node/StaleNodeHandler.java | 4 + .../scm/server/StorageContainerManager.java | 2 +- .../hdds/scm/block/TestBlockManager.java | 3 +- .../hdds/scm/container/MockNodeManager.java | 7 + .../scm/container/SimpleMockNodeManager.java | 15 + .../TestPendingContainerTracker.java | 375 ------------------ .../hdds/scm/node/TestContainerPlacement.java | 3 +- .../scm/pipeline/MockPipelineManager.java | 15 +- .../scm/pipeline/TestPipelineManagerImpl.java | 1 + .../scm/safemode/TestSCMSafeModeManager.java | 3 +- ...estPendingContainerTrackerIntegration.java | 20 +- .../recon/scm/ReconContainerManager.java | 6 +- .../ReconStorageContainerManagerFacade.java | 2 +- .../AbstractReconContainerManagerTest.java | 3 +- 24 files changed, 150 insertions(+), 796 deletions(-) delete mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java delete mode 100644 hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java index 1bd8f2986f70..f36285bfb169 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java @@ -138,8 +138,8 @@ public class ScmConfig extends ReconfigurableConfig { ) private int transactionToDNsCommitMapLimit = 5000000; - @Config(key = "hdds.scm.container.pending-allocation.roll-interval", - defaultValue = "10m", + @Config(key = "hdds.scm.container.pending.allocation.roll.interval", + defaultValue = "5m", type = ConfigType.TIME, tags = { ConfigTag.SCM, ConfigTag.CONTAINER }, description = @@ -149,7 +149,7 @@ public class ScmConfig extends ReconfigurableConfig { "After 2x this interval, allocations that haven't been confirmed via " + "container reports will automatically age out. Default is 10 minutes." ) - private Duration pendingContainerAllocationRollInterval = Duration.ofMinutes(10); + private Duration pendingContainerAllocationRollInterval = Duration.ofMinutes(5); public int getTransactionToDNsCommitMapLimit() { return transactionToDNsCommitMapLimit; 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 85ce9f67a844..a1e58d595222 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 @@ -136,15 +136,15 @@ public List getContainers(ReplicationType type) { @Override public List getContainerIDs(final ContainerID startID, - final int count, - final LifeCycleState state) { + final int count, + final LifeCycleState state) { scmContainerManagerMetrics.incNumListContainersOps(); return containerStateManager.getContainerIDs(state, startID, count); } @Override public List getContainers(final ContainerID startID, - final int count) { + final int count) { scmContainerManagerMetrics.incNumListContainersOps(); return containerStateManager.getContainerInfos(startID, count); } @@ -157,8 +157,8 @@ public List getContainers(final LifeCycleState state) { @Override public List getContainers(final ContainerID startID, - final int count, - final LifeCycleState state) { + final int count, + final LifeCycleState state) { scmContainerManagerMetrics.incNumListContainersOps(); return containerStateManager.getContainerInfos(state, startID, count); } @@ -236,7 +236,7 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner) } private ContainerInfo allocateContainer(final Pipeline pipeline, - final String owner) + final String owner) throws IOException { if (!pipelineManager.hasEnoughSpace(pipeline)) { LOG.debug("Cannot allocate a new container because pipeline {} does not have enough space.", pipeline); @@ -269,15 +269,14 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, } containerStateManager.addContainer(containerInfoBuilder.build()); - scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); - // Record pending allocation - tracks containers scheduled but not yet written pipelineManager.recordPendingAllocation(pipeline, containerID); + scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); return containerStateManager.getContainer(containerID); } @Override public void updateContainerState(final ContainerID cid, - final LifeCycleEvent event) + final LifeCycleEvent event) throws IOException, InvalidStateTransitionException { HddsProtos.ContainerID protoId = cid.getProtobuf(); lock.lock(); @@ -313,7 +312,7 @@ public void updateContainerInfo(final ContainerID cid, ContainerInfoProto contai @Override public void transitionDeletingOrDeletedToTargetState(ContainerID containerID, LifeCycleState targetState) - throws IOException { + throws IOException { HddsProtos.ContainerID proto = containerID.getProtobuf(); lock.lock(); try { @@ -339,7 +338,7 @@ public Set getContainerReplicas(final ContainerID id) @Override public void updateContainerReplica(final ContainerID cid, - final ContainerReplica replica) + final ContainerReplica replica) throws ContainerNotFoundException { if (containerExist(cid)) { containerStateManager.updateContainerReplica(replica); @@ -350,7 +349,7 @@ public void updateContainerReplica(final ContainerID cid, @Override public void removeContainerReplica(final ContainerID cid, - final ContainerReplica replica) + final ContainerReplica replica) throws ContainerNotFoundException, ContainerReplicaNotFoundException { if (containerExist(cid)) { containerStateManager.removeContainerReplica(replica); @@ -421,7 +420,7 @@ private NavigableSet getContainersForOwner( @Override public void notifyContainerReportProcessing(final boolean isFullReport, - final boolean success) { + final boolean success) { if (isFullReport) { if (success) { scmContainerManagerMetrics.incNumContainerReportsProcessedSuccessful(); 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 b9358096d5d7..e34305491e41 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 @@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.ha.SCMContext; import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode; import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer; @@ -179,10 +180,12 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, // Remove from pending tracker when container is added to DN // This container was just confirmed for the first time on this DN // No need to remove on subsequent reports (it's already been removed) - if (container != null && getContainerManager() instanceof ContainerManagerImpl) { - ((ContainerManagerImpl) getContainerManager()) - .getPendingContainerTracker() - .removePendingAllocation(datanodeDetails, cid); + if (container != null) { + PendingContainerTracker tracker = + getNodeManager().getPendingContainerTracker(); + if (tracker != null) { + tracker.removePendingAllocation(datanodeDetails, cid); + } } } if (container == null || ContainerReportValidator 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 ff4eae77f97d..123189887820 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 @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.ha.SCMContext; import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.IncrementalContainerReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventHandler; @@ -89,15 +90,6 @@ protected void processICR(IncrementalContainerReportFromDatanode report, ContainerID id = ContainerID.valueOf(replicaProto.getContainerID()); final ContainerInfo container; try { - // Check if container is already known to this DN before adding - boolean alreadyOnDn = false; - try { - alreadyOnDn = getNodeManager().getContainers(dd).contains(id); - } catch (NodeNotFoundException e) { - // DN not found, treat as not already on DN - getLogger().debug("Datanode not found when checking containers: {}", dd); - } - try { container = getContainerManager().getContainer(id); // Ensure we reuse the same ContainerID instance in containerInfo @@ -112,12 +104,11 @@ protected void processICR(IncrementalContainerReportFromDatanode report, } if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); - - // Remove from pending tracker when container is added to DN - if (!alreadyOnDn && getContainerManager() instanceof ContainerManagerImpl) { - ((ContainerManagerImpl) getContainerManager()) - .getPendingContainerTracker() - .removePendingAllocation(dd, id); + + PendingContainerTracker tracker = + getNodeManager().getPendingContainerTracker(); + if (tracker != null) { + tracker.removePendingAllocation(dd, id); } } success = true; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java deleted file mode 100644 index 4866416b392d..000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/PendingContainerTracker.java +++ /dev/null @@ -1,338 +0,0 @@ -/* - * 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 com.google.common.annotations.VisibleForTesting; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.scm.container.metrics.SCMContainerManagerMetrics; -import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.apache.hadoop.util.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Tracks pending container allocations using a Two Window Tumbling Bucket pattern. - * Similar like HDFS HADOOP-3707. - * - * Two Window Tumbling Bucket for automatic aging and cleanup. - * - * How It Works: - *
  • Each DataNode has two sets: currentWindow and previousWindow
  • - *
  • New allocations go into currentWindow
  • - *
  • Every ROLL_INTERVAL (default 10 minutes): - *
      - *
    • previousWindow = currentWindow (shift)
    • - *
    • currentWindow = new empty set (reset)
    • - *
    • Old previousWindow is discarded (automatic aging)
    • - *
    - *
  • - *
  • When checking pending: return union of currentWindow + previousWindow
  • - * - * - * Example Timeline: - *
    - * Time  | Action                    | CurrentWindow | PreviousWindow | Total Pending
    - * ------+---------------------------+---------------+----------------+--------------
    - * 00:00 | Allocate Container-1      | {C1}          | {}             | {C1}
    - * 00:05 | Allocate Container-2      | {C1, C2}      | {}             | {C1, C2}
    - * 00:10 | [ROLL] Window tumbles     | {}            | {C1, C2}       | {C1, C2}
    - * 00:12 | Allocate Container-3      | {C3}          | {C1, C2}       | {C1, C2, C3}
    - * 00:15 | Report confirms C1        | {C3}          | {C2}           | {C2, C3}
    - * 00:20 | [ROLL] Window tumbles     | {}            | {C3}           | {C3}
    - *       | (C2 aged out if not reported)
    - * 
    - * - */ -public class PendingContainerTracker { - - private static final Logger LOG = LoggerFactory.getLogger(PendingContainerTracker.class); - - /** - * Roll interval in milliseconds. - * Configurable via hdds.scm.container.pending-allocation.roll-interval. - * Default: 10 minutes. - * Containers automatically age out after 2 × rollIntervalMs. - */ - private final long rollIntervalMs; - - /** - * Map of DataNode UUID to TwoWindowBucket. - */ - private final ConcurrentHashMap datanodeBuckets; - - /** - * Maximum container size in bytes. - */ - private final long maxContainerSize; - - /** - * Metrics for tracking pending containers. - */ - private final SCMContainerManagerMetrics metrics; - - /** - * Two-window bucket for a single DataNode. - * Contains current and previous window sets, plus last roll timestamp. - */ - private static class TwoWindowBucket { - private Set currentWindow = ConcurrentHashMap.newKeySet(); - private Set previousWindow = ConcurrentHashMap.newKeySet(); - private long lastRollTime = Time.monotonicNow(); - private final long rollIntervalMs; - - TwoWindowBucket(long rollIntervalMs) { - this.rollIntervalMs = rollIntervalMs; - } - - /** - * Roll the windows: previous = current, current = empty. - * Called when current time exceeds lastRollTime + rollIntervalMs. - */ - synchronized void rollIfNeeded() { - long now = Time.monotonicNow(); - if (now - lastRollTime >= rollIntervalMs) { - // Shift: current becomes previous - previousWindow = currentWindow; - // Reset: new empty current window - currentWindow = ConcurrentHashMap.newKeySet(); - lastRollTime = now; - LOG.debug("Rolled window. Previous window size: {}, Current window reset to empty", previousWindow.size()); - } - } - - /** - * Get union of both windows (all pending containers). - */ - synchronized Set getAllPending() { - Set all = new HashSet<>(); - all.addAll(currentWindow); - all.addAll(previousWindow); - return all; - } - - /** - * Add container to current window. - */ - synchronized boolean add(ContainerID containerID) { - return currentWindow.add(containerID); - } - - /** - * Remove container from both windows. - */ - synchronized boolean remove(ContainerID containerID) { - boolean removedFromCurrent = currentWindow.remove(containerID); - boolean removedFromPrevious = previousWindow.remove(containerID); - return removedFromCurrent || removedFromPrevious; - } - - /** - * Check if either window is non-empty. - */ - synchronized boolean isEmpty() { - return currentWindow.isEmpty() && previousWindow.isEmpty(); - } - - /** - * Get count of all pending containers (union). - */ - synchronized int getCount() { - return getAllPending().size(); - } - } - - public PendingContainerTracker(long maxContainerSize) { - this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes - } - - public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, - SCMContainerManagerMetrics metrics) { - this.datanodeBuckets = new ConcurrentHashMap<>(); - this.maxContainerSize = maxContainerSize; - this.rollIntervalMs = rollIntervalMs; - this.metrics = metrics; - LOG.info("PendingContainerTracker initialized with maxContainerSize={}B, rollInterval={}ms", - maxContainerSize, rollIntervalMs); - } - - /** - * Record a pending container allocation for all DataNodes in the pipeline. - * Container is added to the current window. - * - * @param pipeline The pipeline where container is allocated - * @param containerID The container being allocated - */ - public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { - if (pipeline == null || containerID == null) { - LOG.warn("Ignoring null pipeline or containerID"); - return; - } - - for (DatanodeDetails node : pipeline.getNodes()) { - recordPendingAllocationForDatanode(node, containerID); - } - } - - /** - * Record a pending container allocation for a single DataNode. - * Container is added to the current window. - * - * @param node The DataNode where container is being allocated/replicated - * @param containerID The container being allocated/replicated - */ - public void recordPendingAllocationForDatanode(DatanodeDetails node, ContainerID containerID) { - if (node == null || containerID == null) { - LOG.warn("Ignoring null node or containerID"); - return; - } - - TwoWindowBucket bucket = datanodeBuckets.computeIfAbsent( - node.getUuid(), - k -> new TwoWindowBucket(rollIntervalMs) - ); - - // Roll window if needed before adding - bucket.rollIfNeeded(); - - boolean added = bucket.add(containerID); - LOG.info("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", - containerID, node.getUuidString(), added, bucket.getCount()); - - // Increment metrics counter - if (added && metrics != null) { - metrics.incNumPendingContainersAdded(); - } - } - - /** - * Remove a pending container allocation from a specific DataNode. - * Removes from both current and previous windows. - * Called when container is confirmed. - * - * @param node The DataNode - * @param containerID The container to remove from pending - */ - public void removePendingAllocation(DatanodeDetails node, ContainerID containerID) { - if (node == null || containerID == null) { - return; - } - - TwoWindowBucket bucket = datanodeBuckets.get(node.getUuid()); - if (bucket != null) { - // Roll window if needed before removing - bucket.rollIfNeeded(); - - boolean removed = bucket.remove(containerID); - LOG.info("Removed pending container {} from DataNode {}. Removed={}, Remaining={}", - containerID, node.getUuidString(), removed, bucket.getCount()); - - // Increment metrics counter - if (removed && metrics != null) { - metrics.incNumPendingContainersRemoved(); - } - - // Cleanup empty buckets to prevent memory leak - if (bucket.isEmpty()) { - LOG.info("Cleanup pending bucket"); - datanodeBuckets.remove(node.getUuid(), bucket); - } - } - } - - /** - * Get the total size of pending allocations on a DataNode. - * Returns union of current and previous windows. - * - * @param node The DataNode - * @return Total bytes of pending container allocations - */ - public long getPendingAllocationSize(DatanodeDetails node) { - if (node == null) { - return 0; - } - - TwoWindowBucket bucket = datanodeBuckets.get(node.getUuid()); - LOG.info("Get pending from DataNode {}", - node.getUuidString()); - if (bucket == null) { - LOG.info("Get pending from DataNode {} is null", - node.getUuidString()); - return 0; - } - - // Roll window if needed before querying - bucket.rollIfNeeded(); - - // Each pending container assumes max size - return (long) bucket.getCount() * maxContainerSize; - } - - /** - * Get the set of pending container IDs for a DataNode. - * Returns union of current and previous windows. - * Useful for debugging and monitoring. - * - * @param node The DataNode - * @return Set of pending container IDs - */ - public Set getPendingContainers(DatanodeDetails node) { - if (node == null) { - return Collections.emptySet(); - } - - TwoWindowBucket bucket = datanodeBuckets.get(node.getUuid()); - if (bucket == null) { - return Collections.emptySet(); - } - - bucket.rollIfNeeded(); - return bucket.getAllPending(); - } - - /** - * Get total number of DataNodes with pending allocations. - * - * @return Count of DataNodes - */ - public int getDataNodeCount() { - return datanodeBuckets.size(); - } - - /** - * Get total number of pending containers across all DataNodes. - * Note: Same container on multiple DataNodes is counted once per DataNode. - * The count may include containers from the previous window (up to 10 minutes old). - * - * @return Total pending container count - */ - public long getTotalPendingCount() { - return datanodeBuckets.values().stream() - .mapToLong(TwoWindowBucket::getCount) - .sum(); - } - - @VisibleForTesting - public SCMContainerManagerMetrics getMetrics() { - return metrics; - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java index 32993984285d..ed3670ea6645 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/metrics/SCMContainerManagerMetrics.java @@ -48,10 +48,6 @@ public final class SCMContainerManagerMetrics { private @Metric MutableCounterLong numICRReportsProcessedSuccessful; private @Metric MutableCounterLong numICRReportsProcessedFailed; - // Pending container allocation metrics - private @Metric MutableCounterLong numPendingContainersAdded; - private @Metric MutableCounterLong numPendingContainersRemoved; - private SCMContainerManagerMetrics() { } @@ -145,22 +141,4 @@ public long getNumListContainersOps() { return numListContainerOps.value(); } - // Pending container allocation metrics - - public void incNumPendingContainersAdded() { - this.numPendingContainersAdded.incr(); - } - - public long getNumPendingContainersAdded() { - return numPendingContainersAdded.value(); - } - - public void incNumPendingContainersRemoved() { - this.numPendingContainersRemoved.incr(); - } - - public long getNumPendingContainersRemoved() { - return numPendingContainersRemoved.value(); - } - } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java index da57666cb304..dfae649ebd29 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java @@ -97,6 +97,10 @@ public void onMessage(final DatanodeDetails datanodeDetails, * action. */ LOG.info("A dead datanode is detected. {}", datanodeDetails); + PendingContainerTracker pending = nodeManager.getPendingContainerTracker(); + if (pending != null) { + pending.clearPendingForDatanode(datanodeDetails); + } closeContainers(datanodeDetails, publisher); destroyPipelines(datanodeDetails); 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..09e0dcf3cb68 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 @@ -435,4 +435,15 @@ default void removeNode(DatanodeDetails datanodeDetails) throws NodeNotFoundExce } int openContainerLimit(List datanodes); + + /** + * SCM-side tracker for container allocations not yet reported by datanodes. + */ + PendingContainerTracker getPendingContainerTracker(); + + /** + * True if the node can accept another container of the given size, accounting for + * {@link #getPendingContainerTracker()}. + */ + boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, long containerSize); } 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..f96964e802cd 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 @@ -69,6 +69,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto.ErrorCode; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.ScmConfig; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.VersionInfo; import org.apache.hadoop.hdds.scm.container.ContainerID; @@ -144,6 +145,13 @@ public class SCMNodeManager implements NodeManager { private final NonWritableNodeFilter nonWritableNodeFilter; private final int numContainerPerVolume; + /** + * SCM-side pending container allocations per datanode (not yet in container reports). + */ + private final PendingContainerTracker pendingContainerTracker; + + private final long maxContainerSizeBytes; + /** * Lock used to synchronize some operation in Node manager to ensure a * consistent view of the node state. @@ -211,6 +219,14 @@ public SCMNodeManager( this.scmContext = scmContext; this.sendCommandNotifyMap = new HashMap<>(); this.nonWritableNodeFilter = new NonWritableNodeFilter(conf); + + this.maxContainerSizeBytes = (long) conf.getStorageSize( + ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, + ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); + ScmConfig scmConfig = conf.getObject(ScmConfig.class); + long rollIntervalMs = scmConfig.getPendingContainerAllocationRollInterval().toMillis(); + this.pendingContainerTracker = new PendingContainerTracker( + maxContainerSizeBytes, rollIntervalMs, this.metrics); } @Override @@ -231,6 +247,35 @@ private void unregisterMXBean() { } } + @Override + public PendingContainerTracker getPendingContainerTracker() { + return pendingContainerTracker; + } + + /** + * Effective space check aligned with container allocation: per-disk slot model minus + * SCM pending allocations. + */ + @Override + public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, + long containerSize) { + if (node == null) { + return false; + } + try { + DatanodeInfo datanodeInfo = getDatanodeInfo(node); + if (datanodeInfo == null) { + LOG.warn("DatanodeInfo not found for node {}", node.getUuidString()); + return false; + } + return pendingContainerTracker.hasEffectiveAllocatableSpaceForNewContainer( + node, datanodeInfo, containerSize); + } catch (Exception e) { + LOG.warn("Error checking allocatable space for node {}", node.getUuidString(), e); + return false; + } + } + protected NodeStateManager getNodeStateManager() { return nodeStateManager; } @@ -712,6 +757,7 @@ public void processNodeReport(DatanodeDetails datanodeDetails, datanodeInfo.updateStorageReports(nodeReport.getStorageReportList()); datanodeInfo.updateMetaDataStorageReports(nodeReport. getMetadataStorageReportList()); + pendingContainerTracker.rollWindowsIfNeeded(datanodeDetails); metrics.incNumNodeReportProcessed(); } } catch (NodeNotFoundException e) { @@ -1129,6 +1175,8 @@ private SCMNodeStat getNodeStatInternal(DatanodeDetails datanodeDetails) { freeSpaceToSpare += reportProto.getFreeSpaceToSpare(); reserved += reportProto.getReserved(); } + // SCM-side pending container allocations (not yet in DN reports) count toward committed. + committed += pendingContainerTracker.getPendingAllocationSize(datanodeDetails); return new SCMNodeStat(capacity, used, remaining, committed, freeSpaceToSpare, reserved); } catch (NodeNotFoundException e) { 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..7c8e318152b0 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 @@ -49,6 +49,7 @@ public final class SCMNodeMetrics implements MetricsSource { private @Metric MutableCounterLong numNodeReportProcessingFailed; private @Metric MutableCounterLong numNodeCommandQueueReportProcessed; private @Metric MutableCounterLong numNodeCommandQueueReportProcessingFailed; + // Pending container allocations at SCM (per-DN tracker), not yet on datanodes. private @Metric String textMetric; // Pending container allocations at SCM (per-DN tracker), not yet on datanodes. private @Metric MutableCounterLong numPendingContainersAdded; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java index 60b88e94973e..cb88136d7d88 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java @@ -46,6 +46,10 @@ public StaleNodeHandler(NodeManager nodeManager, @Override public void onMessage(DatanodeDetails datanodeDetails, EventPublisher publisher) { + PendingContainerTracker pending = nodeManager.getPendingContainerTracker(); + if (pending != null) { + pending.clearPendingForDatanode(datanodeDetails); + } Set pipelineIds = nodeManager.getPipelines(datanodeDetails); LOG.info("Datanode {} moved to stale state. Finalizing its pipelines {}", diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 696816b85a24..82c493a03173 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -832,7 +832,7 @@ private void initializeSystemManagers(OzoneConfiguration conf, } else { containerManager = new ContainerManagerImpl(conf, scmHAManager, sequenceIdGen, pipelineManager, scmMetadataStore.getContainerTable(), - containerReplicaPendingOps); + containerReplicaPendingOps, scmNodeManager); } ScmConfig scmConfig = conf.getObject(ScmConfig.class); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java index 45c947cb00a4..8c390cdc1068 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java @@ -154,7 +154,8 @@ void setUp(@TempDir File tempDir) throws Exception { pipelineManager, scmMetadataStore.getContainerTable(), new ContainerReplicaPendingOps( - Clock.system(ZoneId.systemDefault()), null)); + Clock.system(ZoneId.systemDefault()), null), + nodeManager); SCMSafeModeManager safeModeManager = new SCMSafeModeManager(conf, nodeManager, pipelineManager, containerManager, serviceManager, eventQueue, scmContext); SCMConfigurator configurator = new SCMConfigurator(); 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..8174e8bb9c67 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 @@ -545,6 +545,13 @@ public int getPipelinesCount(DatanodeDetails datanodeDetails) { @Override public void addPipeline(Pipeline pipeline) { node2PipelineMap.addPipeline(pipeline); + // Pipeline creation uses DNs that may not be the pre-registered fake nodes; ensure each + // pipeline member has metrics so {@link #getDatanodeInfo} and space checks work. + for (DatanodeDetails dn : pipeline.getNodes()) { + if (nodeMetricMap.get(dn) == null) { + populateNodeMetric(dn, 0); + } + } } /** 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..b176dd62da2f 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()); @@ -445,4 +447,17 @@ public Boolean isNodeRegistered(DatanodeDetails datanodeDetails) { return false; } + @Override + public PendingContainerTracker getPendingContainerTracker() { + if (pendingContainerTracker == null) { + pendingContainerTracker = new PendingContainerTracker(5L * 1024 * 1024 * 1024); + } + return pendingContainerTracker; + } + + @Override + public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, long containerSize) { + return true; + } + } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java deleted file mode 100644 index f7126cf678ba..000000000000 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTracker.java +++ /dev/null @@ -1,375 +0,0 @@ -/* - * 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 org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; - -import java.io.IOException; -import java.util.Set; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; -import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; -import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -/** - * Tests for PendingContainerTracker. - */ -public class TestPendingContainerTracker { - - private static final long MAX_CONTAINER_SIZE = 5L * 1024 * 1024 * 1024; // 5GB - - private PendingContainerTracker tracker; - private Pipeline pipeline; - private DatanodeDetails dn1; - private DatanodeDetails dn2; - private DatanodeDetails dn3; - private ContainerID container1; - private ContainerID container2; - private ContainerID container3; - - @BeforeEach - public void setUp() throws IOException { - tracker = new PendingContainerTracker(MAX_CONTAINER_SIZE); - - // Create a 3-node Ratis pipeline - pipeline = MockPipeline.createPipeline(3); - dn1 = pipeline.getNodes().get(0); - dn2 = pipeline.getNodes().get(1); - dn3 = pipeline.getNodes().get(2); - - container1 = ContainerID.valueOf(1L); - container2 = ContainerID.valueOf(2L); - container3 = ContainerID.valueOf(3L); - } - - @Test - public void testRecordPendingAllocation() { - // Initially no pending containers - assertEquals(0, tracker.getPendingContainers(dn1).size()); - assertEquals(0, tracker.getPendingAllocationSize(dn1)); - - // Record a pending allocation - tracker.recordPendingAllocation(pipeline, container1); - - // All 3 DNs should have the container pending - assertEquals(1, tracker.getPendingContainers(dn1).size()); - assertEquals(1, tracker.getPendingContainers(dn2).size()); - assertEquals(1, tracker.getPendingContainers(dn3).size()); - - // Size should be MAX_CONTAINER_SIZE for each DN - assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); - assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn2)); - assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn3)); - } - - @Test - public void testRecordMultiplePendingAllocations() { - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container2); - tracker.recordPendingAllocation(pipeline, container3); - - // Each DN should have 3 pending containers - assertEquals(3, tracker.getPendingContainers(dn1).size()); - assertEquals(3, tracker.getPendingContainers(dn2).size()); - assertEquals(3, tracker.getPendingContainers(dn3).size()); - - // Size should be 3 × MAX_CONTAINER_SIZE - assertEquals(3 * MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); - } - - @Test - public void testIdempotentRecording() { - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container1); // Duplicate - - // Should still be 1 container (Set deduplication) - assertEquals(1, tracker.getPendingContainers(dn1).size()); - assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); - } - - @Test - public void testRemovePendingAllocation() { - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container2); - - assertEquals(2, tracker.getPendingContainers(dn1).size()); - - // Remove one container from DN1 - tracker.removePendingAllocation(dn1, container1); - - assertEquals(1, tracker.getPendingContainers(dn1).size()); - assertEquals(MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); - - // DN2 and DN3 should still have both containers - assertEquals(2, tracker.getPendingContainers(dn2).size()); - assertEquals(2, tracker.getPendingContainers(dn3).size()); - } - - @Test - public void testRemovePendingAllocationFromPipeline() { - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container2); - - // Remove container1 from all nodes in pipeline - for (DatanodeDetails dn : pipeline.getNodes()) { - tracker.removePendingAllocation(dn, container1); - } - - // All DNs should have only container2 remaining - assertEquals(1, tracker.getPendingContainers(dn1).size()); - assertEquals(1, tracker.getPendingContainers(dn2).size()); - assertEquals(1, tracker.getPendingContainers(dn3).size()); - } - - @Test - public void testRemoveNonExistentContainer() { - tracker.recordPendingAllocation(pipeline, container1); - - // Remove a container that was never added - should not throw exception - tracker.removePendingAllocation(dn1, container2); - - // DN1 should still have container1 - assertEquals(1, tracker.getPendingContainers(dn1).size()); - } - - @Test - public void testGetPendingContainers() { - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container2); - - Set pending = tracker.getPendingContainers(dn1); - - assertEquals(2, pending.size()); - assertThat(pending.contains(container1)); - assertThat(pending.contains(container2)); - - // Returned set should be a copy - modifying it shouldn't affect tracker - pending.add(container3); - assertEquals(2, tracker.getPendingContainers(dn1).size()); // Should still be 2 - } - - @Test - public void testGetPendingContainersForNonExistentDN() { - DatanodeDetails unknownDN = MockDatanodeDetails.randomDatanodeDetails(); - - Set pending = tracker.getPendingContainers(unknownDN); - - assertThat(pending.isEmpty()); - } - - @Test - public void testGetTotalPendingCount() { - assertEquals(0, tracker.getTotalPendingCount()); - - tracker.recordPendingAllocation(pipeline, container1); - - // 1 container × 3 DNs = 3 total pending - assertEquals(3, tracker.getTotalPendingCount()); - - tracker.recordPendingAllocation(pipeline, container2); - - // 2 containers × 3 DNs = 6 total pending - assertEquals(6, tracker.getTotalPendingCount()); - - // Remove from one DN - tracker.removePendingAllocation(dn1, container1); - - // (2 containers × 2 DNs) + (1 container × 1 DN) = 5 total - assertEquals(5, tracker.getTotalPendingCount()); - } - - @Test - public void testConcurrentModification() throws InterruptedException { - // Test thread-safety by having multiple threads add/remove containers - final int numThreads = 10; - final int operationsPerThread = 100; - - Thread[] threads = new Thread[numThreads]; - - for (int i = 0; i < numThreads; i++) { - final int threadId = i; - threads[i] = new Thread(() -> { - for (int j = 0; j < operationsPerThread; j++) { - ContainerID cid = ContainerID.valueOf(threadId * 1000L + j); - tracker.recordPendingAllocation(pipeline, cid); - - if (j % 2 == 0) { - tracker.removePendingAllocation(dn1, cid); - } - } - }); - } - - // Start all threads - for (Thread thread : threads) { - thread.start(); - } - - // Wait for all to finish - for (Thread thread : threads) { - thread.join(); - } - - // Verify no exceptions occurred and counts are reasonable - assertThat(tracker.getTotalPendingCount() >= 0); - assertThat(tracker.getDataNodeCount() <= 3); - } - - @Test - public void testMemoryCleanupOnEmptySet() { - tracker.recordPendingAllocation(pipeline, container1); - - assertEquals(3, tracker.getDataNodeCount()); - - // Remove the only pending container from DN1 - tracker.removePendingAllocation(dn1, container1); - - // DN1 should be removed from the map (memory cleanup) - assertEquals(2, tracker.getDataNodeCount()); - } - - @Test - public void testPendingContainer() { - // Simulate allocation and confirmation flow - - // Allocate 3 containers - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container2); - tracker.recordPendingAllocation(pipeline, container3); - - // Each DN should have 3 pending, 15GB total - assertEquals(3, tracker.getPendingContainers(dn1).size()); - assertEquals(3 * MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); - - // DN1 confirms container1 via container report - tracker.removePendingAllocation(dn1, container1); - - // DN1 now has 2 pending, 10GB - assertEquals(2, tracker.getPendingContainers(dn1).size()); - assertEquals(2 * MAX_CONTAINER_SIZE, tracker.getPendingAllocationSize(dn1)); - - // DN2 and DN3 still have 3 pending - assertEquals(3, tracker.getPendingContainers(dn2).size()); - assertEquals(3, tracker.getPendingContainers(dn3).size()); - - // All DNs eventually confirm all containers - for (DatanodeDetails dn : pipeline.getNodes()) { - tracker.removePendingAllocation(dn, container1); - tracker.removePendingAllocation(dn, container2); - tracker.removePendingAllocation(dn, container3); - } - - // All DNs should have 0 pending - assertEquals(0, tracker.getPendingContainers(dn1).size()); - assertEquals(0, tracker.getPendingContainers(dn2).size()); - assertEquals(0, tracker.getPendingContainers(dn3).size()); - assertEquals(0, tracker.getTotalPendingCount()); - assertEquals(0, tracker.getDataNodeCount()); - } - - @Test - public void testRemoveFromBothWindows() { - // This test verifies that removal works from both current and previous windows - // In general, a container could be in previous window after a roll - - // Add containers - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container2); - - assertEquals(2, tracker.getPendingContainers(dn1).size()); - - // Remove container1 - should work regardless of which window it's in - tracker.removePendingAllocation(dn1, container1); - - assertEquals(1, tracker.getPendingContainers(dn1).size()); - - Set pending = tracker.getPendingContainers(dn1); - assertFalse(pending.contains(container1)); - assertThat(pending.contains(container2)); - } - - @Test - public void testUnionOfBothWindows() { - // This test verifies the two-window concept: - // getPendingContainers should return union of current + previous windows - - // Add container1 - tracker.recordPendingAllocation(pipeline, container1); - - assertEquals(1, tracker.getPendingContainers(dn1).size()); - Set pending1 = tracker.getPendingContainers(dn1); - assertThat(pending1.contains(container1)); - - // Add container2 - should be in same window initially - tracker.recordPendingAllocation(pipeline, container2); - - assertEquals(2, tracker.getPendingContainers(dn1).size()); - Set pending2 = tracker.getPendingContainers(dn1); - assertThat(pending2.contains(container1)); - assertThat(pending2.contains(container2)); - - // Both containers should be in the union - assertEquals(2, pending2.size()); - } - - @Test - public void testIdempotencyAcrossWindows() { - // Recording same container multiple times should only count it once - // This should work even if it spans windows - - tracker.recordPendingAllocation(pipeline, container1); - assertEquals(1, tracker.getPendingContainers(dn1).size()); - - // Record again - should still be 1 (idempotency via Set) - tracker.recordPendingAllocation(pipeline, container1); - assertEquals(1, tracker.getPendingContainers(dn1).size()); - - // Add different container - tracker.recordPendingAllocation(pipeline, container2); - assertEquals(2, tracker.getPendingContainers(dn1).size()); - - // Record container1 again - tracker.recordPendingAllocation(pipeline, container1); - assertEquals(2, tracker.getPendingContainers(dn1).size()); // Still 2, not 3 - } - - @Test - public void testExplicitRemoval() { - - tracker.recordPendingAllocation(pipeline, container1); - tracker.recordPendingAllocation(pipeline, container2); - tracker.recordPendingAllocation(pipeline, container3); - - assertEquals(3, tracker.getPendingContainers(dn1).size()); - - // Simulate container report confirms container1 and container2 - tracker.removePendingAllocation(dn1, container1); - tracker.removePendingAllocation(dn1, container2); - - // Immediately reflects the removal (doesn't wait for aging) - assertEquals(1, tracker.getPendingContainers(dn1).size()); - - Set pending = tracker.getPendingContainers(dn1); - assertEquals(1, pending.size()); - assertThat(pending.contains(container3)); - } -} 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..fa9e36356fca 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 @@ -95,6 +95,7 @@ public class TestContainerPlacement { private SequenceIdGenerator sequenceIdGen; private OzoneConfiguration conf; private PipelineManager pipelineManager; + private NodeManager nodeManager; @BeforeEach public void setUp() throws Exception { @@ -104,7 +105,7 @@ public void setUp() throws Exception { scmhaManager = SCMHAManagerStub.getInstance(true); sequenceIdGen = new SequenceIdGenerator( conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore)); - NodeManager nodeManager = new MockNodeManager(true, 10); + nodeManager = new MockNodeManager(true, 10); pipelineManager = new MockPipelineManager(dbStore, scmhaManager, nodeManager); pipelineManager.createPipeline(RatisReplicationConfig.getInstance( 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 abcb916a16ac..8054d367b87a 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 @@ -37,6 +37,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.scm.ha.SCMHAManager; import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.utils.db.CodecException; import org.apache.hadoop.hdds.utils.db.DBStore; @@ -335,14 +336,16 @@ public boolean isPipelineCreationFrozen() { @Override public boolean hasEnoughSpace(Pipeline pipeline) { - return false; + for (DatanodeDetails node : pipeline.getNodes()) { + if (!nodeManager.hasSpaceForNewContainerAllocation(node)) { + return false; + } + } + return true; } @Override public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { - for (DatanodeDetails dn : pipeline.getNodes()) { - nodeManager.recordPendingAllocationForDatanode(dn.getID(), containerID); - } } @Override @@ -357,7 +360,7 @@ public SCMPipelineMetrics getMetrics() { } @Override - public org.apache.hadoop.hdds.scm.node.DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails) { - return null; + public DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails) { + return nodeManager.getDatanodeInfo(datanodeDetails); } } 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..c305d76e5baa 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 @@ -76,6 +76,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; +import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerInfo; diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java index fdf38a7a67ca..a011d87cd17b 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java @@ -584,7 +584,8 @@ public void testContainerSafeModeRuleEC(int data, int parity) throws Exception { ContainerManager containerManager = new ContainerManagerImpl(config, SCMHAManagerStub.getInstance(true), null, pipelineManager, scmMetadataStore.getContainerTable(), - new ContainerReplicaPendingOps(Clock.system(ZoneId.systemDefault()), null)); + new ContainerReplicaPendingOps(Clock.system(ZoneId.systemDefault()), null), + nodeManager); scmSafeModeManager = new SCMSafeModeManager(config, nodeManager, pipelineManager, containerManager, serviceManager, queue, scmContext); 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 index f2a1280a64b1..ac53f43a9d52 100644 --- 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 @@ -31,7 +31,9 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; -import org.apache.hadoop.hdds.scm.container.metrics.SCMContainerManagerMetrics; +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.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.ozone.MiniOzoneCluster; @@ -60,7 +62,7 @@ public class TestPendingContainerTrackerIntegration { private OzoneClient client; private ContainerManager containerManager; private PendingContainerTracker pendingTracker; - private SCMContainerManagerMetrics metrics; + private SCMNodeMetrics metrics; private OzoneBucket bucket; @BeforeEach @@ -90,23 +92,17 @@ public void setup() throws Exception { // Create bucket for testing bucket = TestDataUtil.createVolumeAndBucket(client); - // Get the pending tracker - if (containerManager instanceof ContainerManagerImpl) { - pendingTracker = ((ContainerManagerImpl) containerManager) - .getPendingContainerTracker(); - assertNotNull(pendingTracker, "PendingContainerTracker should be initialized"); - } + SCMNodeManager nodeManager = (SCMNodeManager) scm.getScmNodeManager(); + assertNotNull(nodeManager); + pendingTracker = nodeManager.getPendingContainerTracker(); + assertNotNull(pendingTracker, "PendingContainerTracker should be initialized"); metrics = pendingTracker.getMetrics(); - // metrics = SCMContainerManagerMetrics.create(); LOG.info("Test setup complete - ICR interval: 5s, Heartbeat interval: 1s"); } @AfterEach public void cleanup() throws Exception { - if (metrics != null) { - metrics.unRegister(); - } if (client != null) { client.close(); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java index 586aad5fd68f..31488c6d4a03 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps; import org.apache.hadoop.hdds.scm.ha.SCMHAManager; import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator; +import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.utils.db.DBStore; @@ -85,10 +86,11 @@ public ReconContainerManager( ReconContainerMetadataManager reconContainerMetadataManager, SCMHAManager scmhaManager, SequenceIdGenerator sequenceIdGen, - ContainerReplicaPendingOps pendingOps) + ContainerReplicaPendingOps pendingOps, + NodeManager nodeManager) throws IOException { super(conf, scmhaManager, sequenceIdGen, pipelineManager, containerStore, - pendingOps); + pendingOps, nodeManager); this.scmClient = scm; this.pipelineManager = pipelineManager; this.containerHealthSchemaManager = containerHealthSchemaManager; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java index 278bac0011dc..46926b7d4fe0 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java @@ -246,7 +246,7 @@ public ReconStorageContainerManagerFacade(OzoneConfiguration conf, pipelineManager, scmServiceProvider, containerHealthSchemaManager, reconContainerMetadataManager, - scmhaManager, sequenceIdGen, pendingOps); + scmhaManager, sequenceIdGen, pendingOps, nodeManager); this.scmServiceProvider = scmServiceProvider; this.isSyncDataFromSCMRunning = new AtomicBoolean(); this.containerCountBySizeDao = containerCountBySizeDao; diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java index 33e20413bfd6..0900fd2bb96f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java @@ -116,7 +116,8 @@ public void setUp(@TempDir File tempDir) throws Exception { mock(ReconContainerMetadataManager.class), scmhaManager, sequenceIdGen, - pendingOps); + pendingOps, + nodeManager); } @AfterEach From 978290d4cdd29ce0fc5020ede8da535e95e11d93 Mon Sep 17 00:00:00 2001 From: ashishk Date: Sun, 5 Apr 2026 01:52:31 +0530 Subject: [PATCH 05/22] Move declaration to top --- .../src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java | 2 +- .../apache/hadoop/hdds/scm/container/ContainerManagerImpl.java | 2 +- .../org/apache/hadoop/hdds/scm/container/MockNodeManager.java | 1 - 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java index f36285bfb169..3771819d1ce0 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java @@ -147,7 +147,7 @@ public class ScmConfig extends ReconfigurableConfig { "Pending container allocations are tracked in a two-window tumbling bucket " + "pattern. Each window has this duration. " + "After 2x this interval, allocations that haven't been confirmed via " + - "container reports will automatically age out. Default is 10 minutes." + "container reports will automatically age out. Default is 5 minutes." ) private Duration pendingContainerAllocationRollInterval = Duration.ofMinutes(5); 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 a1e58d595222..af6f150a51b6 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 @@ -236,7 +236,7 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner) } private ContainerInfo allocateContainer(final Pipeline pipeline, - final String owner) + final String owner) throws IOException { if (!pipelineManager.hasEnoughSpace(pipeline)) { LOG.debug("Cannot allocate a new container because pipeline {} does not have enough space.", 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 8174e8bb9c67..db8e7b48c180 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 @@ -116,7 +116,6 @@ public class MockNodeManager implements NodeManager { private int numPipelinePerDatanode; private PendingContainerTracker pendingContainerTracker; private final OzoneConfiguration conf = new OzoneConfiguration(); - { this.healthyNodes = new LinkedList<>(); this.staleNodes = new LinkedList<>(); From 0d0163d0045f75653a2c3d398808ac7e74f04dfd Mon Sep 17 00:00:00 2001 From: ashishk Date: Tue, 7 Apr 2026 12:19:28 +0530 Subject: [PATCH 06/22] Fix review comment and test --- .../hadoop/hdds/scm/server/SCMClientProtocolServer.java | 7 +++++++ .../apache/hadoop/hdds/scm/container/MockNodeManager.java | 7 ------- 2 files changed, 7 insertions(+), 7 deletions(-) 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 156eed688d81..38cacb1c6086 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 @@ -255,6 +255,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 db8e7b48c180..b9ac0e874dd8 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 @@ -544,13 +544,6 @@ public int getPipelinesCount(DatanodeDetails datanodeDetails) { @Override public void addPipeline(Pipeline pipeline) { node2PipelineMap.addPipeline(pipeline); - // Pipeline creation uses DNs that may not be the pre-registered fake nodes; ensure each - // pipeline member has metrics so {@link #getDatanodeInfo} and space checks work. - for (DatanodeDetails dn : pipeline.getNodes()) { - if (nodeMetricMap.get(dn) == null) { - populateNodeMetric(dn, 0); - } - } } /** From f383238750f5ea1dc118d717d7dfc681859b3b6d Mon Sep 17 00:00:00 2001 From: ashishk Date: Wed, 8 Apr 2026 13:41:56 +0530 Subject: [PATCH 07/22] Fix review comments --- .../hadoop/hdds/scm/pipeline/MockPipelineManager.java | 6 ------ .../TestPendingContainerTrackerIntegration.java | 10 +++++----- 2 files changed, 5 insertions(+), 11 deletions(-) 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 8054d367b87a..2889fdcb6cc2 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 @@ -37,7 +37,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.scm.ha.SCMHAManager; import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition; -import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.utils.db.CodecException; import org.apache.hadoop.hdds.utils.db.DBStore; @@ -358,9 +357,4 @@ public int openContainerLimit(List datanodes) { public SCMPipelineMetrics getMetrics() { return null; } - - @Override - public DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails) { - return nodeManager.getDatanodeInfo(datanodeDetails); - } } 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 index ac53f43a9d52..dcdaa63dd3d6 100644 --- 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 @@ -141,15 +141,15 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { LOG.info("DataNode {} has {} bytes pending", dn.getUuidString(), pendingSize); Set pendingContainers = pendingTracker.getPendingContainers(dn); - assertThat(pendingContainers.contains(container.containerID())); + assertThat(pendingContainers).contains(container.containerID()); } } - assertThat(!nodesWithPending.isEmpty()); + assertThat(nodesWithPending).isNotEmpty(); // Verify metrics increased long afterAdded = metrics.getNumPendingContainersAdded(); - assertThat(afterAdded > initialAdded); + assertThat(afterAdded).isGreaterThan(initialAdded); LOG.info("Pending tracked successfully. Waiting for ICR to remove pending..."); @@ -182,12 +182,12 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { // Verify all pending containers removed for (DatanodeDetails dn : nodesWithPending) { Set pendingContainers = pendingTracker.getPendingContainers(dn); - assertThat(!pendingContainers.contains(container.containerID())); + assertThat(pendingContainers).doesNotContain(container.containerID()); } // Verify remove metrics increased long afterRemoved = metrics.getNumPendingContainersRemoved(); - assertThat(afterRemoved > initialRemoved); + assertThat(afterRemoved).isGreaterThan(initialRemoved); LOG.info("After added = " + afterAdded); From 6ee5248abe6cb7eec8fb5d12df4e4c2e3c3d4f9b Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 20 Apr 2026 12:42:32 +0530 Subject: [PATCH 08/22] Rebase and update fix --- .../scm/container/ContainerManagerImpl.java | 2 + .../hadoop/hdds/scm/node/DeadNodeHandler.java | 4 -- .../hadoop/hdds/scm/node/NodeManager.java | 2 +- .../hadoop/hdds/scm/node/SCMNodeManager.java | 11 ++---- .../hadoop/hdds/scm/node/SCMNodeMetrics.java | 8 ++++ .../hdds/scm/node/StaleNodeHandler.java | 4 -- .../scm/container/SimpleMockNodeManager.java | 5 ++- ...estPendingContainerTrackerIntegration.java | 39 ++++++++----------- 8 files changed, 33 insertions(+), 42 deletions(-) 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 af6f150a51b6..4c3af526f5be 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 @@ -271,6 +271,8 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, containerStateManager.addContainer(containerInfoBuilder.build()); pipelineManager.recordPendingAllocation(pipeline, containerID); scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); + pipelineManager.recordPendingAllocation(pipeline, containerID); + return containerStateManager.getContainer(containerID); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java index dfae649ebd29..da57666cb304 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java @@ -97,10 +97,6 @@ public void onMessage(final DatanodeDetails datanodeDetails, * action. */ LOG.info("A dead datanode is detected. {}", datanodeDetails); - PendingContainerTracker pending = nodeManager.getPendingContainerTracker(); - if (pending != null) { - pending.clearPendingForDatanode(datanodeDetails); - } closeContainers(datanodeDetails, publisher); destroyPipelines(datanodeDetails); 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 09e0dcf3cb68..5eb5dbe7c5aa 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 @@ -445,5 +445,5 @@ default void removeNode(DatanodeDetails datanodeDetails) throws NodeNotFoundExce * True if the node can accept another container of the given size, accounting for * {@link #getPendingContainerTracker()}. */ - boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, long containerSize); + boolean hasSpaceForNewContainerAllocation(DatanodeDetails node); } 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 f96964e802cd..01ed43d1ed81 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 @@ -257,11 +257,8 @@ public PendingContainerTracker getPendingContainerTracker() { * SCM pending allocations. */ @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, - long containerSize) { - if (node == null) { - return false; - } + public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node) { + Objects.requireNonNull(node, "node==null"); try { DatanodeInfo datanodeInfo = getDatanodeInfo(node); if (datanodeInfo == null) { @@ -269,7 +266,7 @@ public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, return false; } return pendingContainerTracker.hasEffectiveAllocatableSpaceForNewContainer( - node, datanodeInfo, containerSize); + node, datanodeInfo); } catch (Exception e) { LOG.warn("Error checking allocatable space for node {}", node.getUuidString(), e); return false; @@ -1175,8 +1172,6 @@ private SCMNodeStat getNodeStatInternal(DatanodeDetails datanodeDetails) { freeSpaceToSpare += reportProto.getFreeSpaceToSpare(); reserved += reportProto.getReserved(); } - // SCM-side pending container allocations (not yet in DN reports) count toward committed. - committed += pendingContainerTracker.getPendingAllocationSize(datanodeDetails); return new SCMNodeStat(capacity, used, remaining, committed, freeSpaceToSpare, reserved); } catch (NodeNotFoundException e) { 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 7c8e318152b0..ece8fcc26cc1 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 @@ -137,6 +137,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/node/StaleNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java index cb88136d7d88..60b88e94973e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StaleNodeHandler.java @@ -46,10 +46,6 @@ public StaleNodeHandler(NodeManager nodeManager, @Override public void onMessage(DatanodeDetails datanodeDetails, EventPublisher publisher) { - PendingContainerTracker pending = nodeManager.getPendingContainerTracker(); - if (pending != null) { - pending.clearPendingForDatanode(datanodeDetails); - } Set pipelineIds = nodeManager.getPipelines(datanodeDetails); LOG.info("Datanode {} moved to stale state. Finalizing its pipelines {}", 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 b176dd62da2f..ae89d696b999 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 @@ -449,14 +449,15 @@ public Boolean isNodeRegistered(DatanodeDetails datanodeDetails) { @Override public PendingContainerTracker getPendingContainerTracker() { + int rollIntervalMs = 5 * 60 * 1000; if (pendingContainerTracker == null) { - pendingContainerTracker = new PendingContainerTracker(5L * 1024 * 1024 * 1024); + pendingContainerTracker = new PendingContainerTracker(5L * 1024 * 1024 * 1024, rollIntervalMs, null); } return pendingContainerTracker; } @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, long containerSize) { + public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node) { return true; } 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 index dcdaa63dd3d6..83053c5e1f1a 100644 --- 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 @@ -20,11 +20,11 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import java.util.ArrayList; import java.util.List; -import java.util.Set; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -135,13 +135,12 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { // Verify pending containers are tracked for all nodes in pipeline List nodesWithPending = new ArrayList<>(); for (DatanodeDetails dn : pipeline.getNodes()) { - long pendingSize = pendingTracker.getPendingAllocationSize(dn); - if (pendingSize > 0) { + long pendingCount = pendingTracker.getPendingContainerCount(dn); + if (pendingCount > 0) { nodesWithPending.add(dn); - LOG.info("DataNode {} has {} bytes pending", dn.getUuidString(), pendingSize); + LOG.info("DataNode {} has {} pending containers", dn.getUuidString(), pendingCount); - Set pendingContainers = pendingTracker.getPendingContainers(dn); - assertThat(pendingContainers).contains(container.containerID()); + assertThat(pendingTracker.containsPendingContainer(dn, container.containerID())); } } @@ -155,7 +154,7 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { // Write a key String keyName = "testKey1"; - byte[] data = "Hello Ozone - Testing Pending Container Tracker".getBytes(UTF_8); + byte[] data = "Testing Pending Container Tracker".getBytes(UTF_8); LOG.info("Writing key: {}", keyName); try (OzoneOutputStream out = bucket.createKey(keyName, data.length, @@ -168,8 +167,7 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { // Wait for ICRs to be sent GenericTestUtils.waitFor(() -> { for (DatanodeDetails dn : nodesWithPending) { - Set pendingContainers = pendingTracker.getPendingContainers(dn); - if (pendingContainers.contains(container.containerID())) { + if (pendingTracker.containsPendingContainer(dn, container.containerID())) { LOG.info("Still waiting for ICR from DN {}", dn.getUuidString()); return false; } @@ -181,8 +179,7 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { // Verify all pending containers removed for (DatanodeDetails dn : nodesWithPending) { - Set pendingContainers = pendingTracker.getPendingContainers(dn); - assertThat(pendingContainers).doesNotContain(container.containerID()); + assertFalse(pendingTracker.containsPendingContainer(dn, container.containerID())); } // Verify remove metrics increased @@ -207,20 +204,16 @@ public void testIdempotentPendingTracking() throws Exception { DatanodeDetails firstNode = pipeline.getFirstNode(); // Record initial state - long initialSize = pendingTracker.getPendingAllocationSize(firstNode); - int initialCount = pendingTracker.getPendingContainers(firstNode).size(); - - LOG.info("Initial pending state: size={}, count={}", initialSize, initialCount); - + long initialCount = pendingTracker.getPendingContainerCount(firstNode); + + LOG.info("Initial pending state: count={}", initialCount); + // Try adding the same container again (simulates retry or duplicate allocation) pendingTracker.recordPendingAllocationForDatanode(firstNode, container.containerID()); - - long afterSize = pendingTracker.getPendingAllocationSize(firstNode); - int afterCount = pendingTracker.getPendingContainers(firstNode).size(); - - // Size and count should remain the same (idempotent) - assertEquals(initialSize, afterSize, - "Pending size should not change when adding duplicate container"); + + long afterCount = pendingTracker.getPendingContainerCount(firstNode); + + // Count should remain the same (idempotent) assertEquals(initialCount, afterCount, "Pending count should not change when adding duplicate container"); From c46e89ac015a3774166014f5163608c7556bb86c Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 20 Apr 2026 14:34:26 +0530 Subject: [PATCH 09/22] Refactor code --- .../apache/hadoop/hdds/scm/node/NodeManager.java | 13 +++++++++++++ .../apache/hadoop/hdds/scm/node/SCMNodeManager.java | 8 ++++++++ .../apache/hadoop/hdds/scm/node/SCMNodeMetrics.java | 1 - .../hadoop/hdds/scm/pipeline/PipelineManager.java | 2 ++ .../hdds/scm/server/StorageContainerManager.java | 2 +- .../hadoop/hdds/scm/block/TestBlockManager.java | 3 +-- .../hdds/scm/safemode/TestSCMSafeModeManager.java | 3 +-- .../ozone/recon/scm/ReconContainerManager.java | 6 ++---- .../scm/ReconStorageContainerManagerFacade.java | 2 +- .../scm/AbstractReconContainerManagerTest.java | 3 +-- 10 files changed, 30 insertions(+), 13 deletions(-) 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 5eb5dbe7c5aa..91cd8cdb0feb 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 @@ -446,4 +446,17 @@ default void removeNode(DatanodeDetails datanodeDetails) throws NodeNotFoundExce * {@link #getPendingContainerTracker()}. */ boolean hasSpaceForNewContainerAllocation(DatanodeDetails node); + + /** + * Records a pending container allocation for {@code node} so that subsequent + * space checks via {@link #hasSpaceForNewContainerAllocation} account for the + * in-flight allocation before the datanode sends an ICR. + * + * @param node the datanode that will receive the new container replica + * @param containerID the container being allocated + */ + default void recordPendingAllocationForDatanode(DatanodeDetails node, + ContainerID containerID) { + getPendingContainerTracker().recordPendingAllocationForDatanode(node, containerID); + } } 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 01ed43d1ed81..f054d9c7d8ac 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 @@ -273,6 +273,14 @@ public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node) { } } + @Override + public void recordPendingAllocationForDatanode(DatanodeDetails node, + ContainerID containerID) { + Objects.requireNonNull(node, "node==null"); + Objects.requireNonNull(containerID, "containerID==null"); + pendingContainerTracker.recordPendingAllocationForDatanode(node, containerID); + } + protected NodeStateManager getNodeStateManager() { return nodeStateManager; } 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 ece8fcc26cc1..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 @@ -49,7 +49,6 @@ public final class SCMNodeMetrics implements MetricsSource { private @Metric MutableCounterLong numNodeReportProcessingFailed; private @Metric MutableCounterLong numNodeCommandQueueReportProcessed; private @Metric MutableCounterLong numNodeCommandQueueReportProcessingFailed; - // Pending container allocations at SCM (per-DN tracker), not yet on datanodes. private @Metric String textMetric; // Pending container allocations at SCM (per-DN tracker), not yet on datanodes. private @Metric MutableCounterLong numPendingContainersAdded; 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..e4c4646a33ed 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 @@ -232,6 +232,8 @@ void reinitialize(Table pipelineStore) */ boolean hasEnoughSpace(Pipeline pipeline); + void recordPendingAllocation(Pipeline pipeline, ContainerID containerID); + int openContainerLimit(List datanodes); /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 82c493a03173..696816b85a24 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -832,7 +832,7 @@ private void initializeSystemManagers(OzoneConfiguration conf, } else { containerManager = new ContainerManagerImpl(conf, scmHAManager, sequenceIdGen, pipelineManager, scmMetadataStore.getContainerTable(), - containerReplicaPendingOps, scmNodeManager); + containerReplicaPendingOps); } ScmConfig scmConfig = conf.getObject(ScmConfig.class); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java index 8c390cdc1068..45c947cb00a4 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java @@ -154,8 +154,7 @@ void setUp(@TempDir File tempDir) throws Exception { pipelineManager, scmMetadataStore.getContainerTable(), new ContainerReplicaPendingOps( - Clock.system(ZoneId.systemDefault()), null), - nodeManager); + Clock.system(ZoneId.systemDefault()), null)); SCMSafeModeManager safeModeManager = new SCMSafeModeManager(conf, nodeManager, pipelineManager, containerManager, serviceManager, eventQueue, scmContext); SCMConfigurator configurator = new SCMConfigurator(); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java index a011d87cd17b..fdf38a7a67ca 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java @@ -584,8 +584,7 @@ public void testContainerSafeModeRuleEC(int data, int parity) throws Exception { ContainerManager containerManager = new ContainerManagerImpl(config, SCMHAManagerStub.getInstance(true), null, pipelineManager, scmMetadataStore.getContainerTable(), - new ContainerReplicaPendingOps(Clock.system(ZoneId.systemDefault()), null), - nodeManager); + new ContainerReplicaPendingOps(Clock.system(ZoneId.systemDefault()), null)); scmSafeModeManager = new SCMSafeModeManager(config, nodeManager, pipelineManager, containerManager, serviceManager, queue, scmContext); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java index 31488c6d4a03..586aad5fd68f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconContainerManager.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps; import org.apache.hadoop.hdds.scm.ha.SCMHAManager; import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator; -import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.utils.db.DBStore; @@ -86,11 +85,10 @@ public ReconContainerManager( ReconContainerMetadataManager reconContainerMetadataManager, SCMHAManager scmhaManager, SequenceIdGenerator sequenceIdGen, - ContainerReplicaPendingOps pendingOps, - NodeManager nodeManager) + ContainerReplicaPendingOps pendingOps) throws IOException { super(conf, scmhaManager, sequenceIdGen, pipelineManager, containerStore, - pendingOps, nodeManager); + pendingOps); this.scmClient = scm; this.pipelineManager = pipelineManager; this.containerHealthSchemaManager = containerHealthSchemaManager; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java index 46926b7d4fe0..278bac0011dc 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java @@ -246,7 +246,7 @@ public ReconStorageContainerManagerFacade(OzoneConfiguration conf, pipelineManager, scmServiceProvider, containerHealthSchemaManager, reconContainerMetadataManager, - scmhaManager, sequenceIdGen, pendingOps, nodeManager); + scmhaManager, sequenceIdGen, pendingOps); this.scmServiceProvider = scmServiceProvider; this.isSyncDataFromSCMRunning = new AtomicBoolean(); this.containerCountBySizeDao = containerCountBySizeDao; diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java index 0900fd2bb96f..33e20413bfd6 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/scm/AbstractReconContainerManagerTest.java @@ -116,8 +116,7 @@ public void setUp(@TempDir File tempDir) throws Exception { mock(ReconContainerMetadataManager.class), scmhaManager, sequenceIdGen, - pendingOps, - nodeManager); + pendingOps); } @AfterEach From f12aba76d6a192d6e54e531d1f6e70895a45ea38 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 20 Apr 2026 14:43:30 +0530 Subject: [PATCH 10/22] Move member to local variable --- .../apache/hadoop/hdds/scm/node/TestContainerPlacement.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 fa9e36356fca..4dbe79fc1351 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 @@ -95,7 +95,6 @@ public class TestContainerPlacement { private SequenceIdGenerator sequenceIdGen; private OzoneConfiguration conf; private PipelineManager pipelineManager; - private NodeManager nodeManager; @BeforeEach public void setUp() throws Exception { @@ -105,7 +104,7 @@ public void setUp() throws Exception { scmhaManager = SCMHAManagerStub.getInstance(true); sequenceIdGen = new SequenceIdGenerator( conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore)); - nodeManager = new MockNodeManager(true, 10); + NodeManager nodeManager = new MockNodeManager(true, 10); pipelineManager = new MockPipelineManager(dbStore, scmhaManager, nodeManager); pipelineManager.createPipeline(RatisReplicationConfig.getInstance( From 203b856c36234d88dab261fcdd7c3d3f5a42e84e Mon Sep 17 00:00:00 2001 From: ashishk Date: Wed, 29 Apr 2026 12:55:10 +0530 Subject: [PATCH 11/22] Rebase and fix review comments --- .../org/apache/hadoop/hdds/scm/ScmConfig.java | 21 ---- .../apache/hadoop/hdds/scm/ScmConfigKeys.java | 5 + .../src/main/resources/ozone-default.xml | 11 ++ .../scm/container/ContainerManagerImpl.java | 22 ++-- .../scm/container/ContainerReportHandler.java | 12 +- .../IncrementalContainerReportHandler.java | 8 +- .../hadoop/hdds/scm/node/NodeManager.java | 19 ---- .../hdds/scm/node/NodeStateManager.java | 8 +- .../scm/node/PendingContainerTracker.java | 6 + .../hadoop/hdds/scm/node/SCMNodeManager.java | 51 +-------- .../hdds/scm/pipeline/PipelineManager.java | 2 - .../hdds/scm/container/MockNodeManager.java | 6 + .../scm/container/SimpleMockNodeManager.java | 6 - .../scm/pipeline/MockPipelineManager.java | 5 +- .../scm/pipeline/TestPipelineManagerImpl.java | 1 - ...estPendingContainerTrackerIntegration.java | 106 ++++++------------ 16 files changed, 92 insertions(+), 197 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java index 3771819d1ce0..9d3d7ef85cbe 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java @@ -138,31 +138,10 @@ public class ScmConfig extends ReconfigurableConfig { ) private int transactionToDNsCommitMapLimit = 5000000; - @Config(key = "hdds.scm.container.pending.allocation.roll.interval", - defaultValue = "5m", - type = ConfigType.TIME, - tags = { ConfigTag.SCM, ConfigTag.CONTAINER }, - description = - "Time interval for rolling the pending container allocation window. " + - "Pending container allocations are tracked in a two-window tumbling bucket " + - "pattern. Each window has this duration. " + - "After 2x this interval, allocations that haven't been confirmed via " + - "container reports will automatically age out. Default is 5 minutes." - ) - private Duration pendingContainerAllocationRollInterval = Duration.ofMinutes(5); - public int getTransactionToDNsCommitMapLimit() { return transactionToDNsCommitMapLimit; } - public Duration getPendingContainerAllocationRollInterval() { - return pendingContainerAllocationRollInterval; - } - - public void setPendingContainerAllocationRollInterval(Duration duration) { - this.pendingContainerAllocationRollInterval = duration; - } - public Duration getBlockDeletionInterval() { return blockDeletionInterval; } 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 a90bacc9fee2..a2fcc555f18a 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 2cb1f52ede15..e8581661c5c0 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 4c3af526f5be..e87bd193b000 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 @@ -136,15 +136,15 @@ public List getContainers(ReplicationType type) { @Override public List getContainerIDs(final ContainerID startID, - final int count, - final LifeCycleState state) { + final int count, + final LifeCycleState state) { scmContainerManagerMetrics.incNumListContainersOps(); return containerStateManager.getContainerIDs(state, startID, count); } @Override public List getContainers(final ContainerID startID, - final int count) { + final int count) { scmContainerManagerMetrics.incNumListContainersOps(); return containerStateManager.getContainerInfos(startID, count); } @@ -157,8 +157,8 @@ public List getContainers(final LifeCycleState state) { @Override public List getContainers(final ContainerID startID, - final int count, - final LifeCycleState state) { + final int count, + final LifeCycleState state) { scmContainerManagerMetrics.incNumListContainersOps(); return containerStateManager.getContainerInfos(state, startID, count); } @@ -271,14 +271,12 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, containerStateManager.addContainer(containerInfoBuilder.build()); pipelineManager.recordPendingAllocation(pipeline, containerID); scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); - pipelineManager.recordPendingAllocation(pipeline, containerID); - return containerStateManager.getContainer(containerID); } @Override public void updateContainerState(final ContainerID cid, - final LifeCycleEvent event) + final LifeCycleEvent event) throws IOException, InvalidStateTransitionException { HddsProtos.ContainerID protoId = cid.getProtobuf(); lock.lock(); @@ -314,7 +312,7 @@ public void updateContainerInfo(final ContainerID cid, ContainerInfoProto contai @Override public void transitionDeletingOrDeletedToTargetState(ContainerID containerID, LifeCycleState targetState) - throws IOException { + throws IOException { HddsProtos.ContainerID proto = containerID.getProtobuf(); lock.lock(); try { @@ -340,7 +338,7 @@ public Set getContainerReplicas(final ContainerID id) @Override public void updateContainerReplica(final ContainerID cid, - final ContainerReplica replica) + final ContainerReplica replica) throws ContainerNotFoundException { if (containerExist(cid)) { containerStateManager.updateContainerReplica(replica); @@ -351,7 +349,7 @@ public void updateContainerReplica(final ContainerID cid, @Override public void removeContainerReplica(final ContainerID cid, - final ContainerReplica replica) + final ContainerReplica replica) throws ContainerNotFoundException, ContainerReplicaNotFoundException { if (containerExist(cid)) { containerStateManager.removeContainerReplica(replica); @@ -422,7 +420,7 @@ private NavigableSet getContainersForOwner( @Override public void notifyContainerReportProcessing(final boolean isFullReport, - final boolean success) { + final boolean success) { if (isFullReport) { if (success) { scmContainerManagerMetrics.incNumContainerReportsProcessedSuccessful(); 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 e34305491e41..dc1f0b586d77 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 @@ -176,17 +176,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 - // No need to remove on subsequent reports (it's already been removed) - if (container != null) { - PendingContainerTracker tracker = - getNodeManager().getPendingContainerTracker(); - if (tracker != null) { - tracker.removePendingAllocation(datanodeDetails, cid); - } - } + PendingContainerTracker tracker = getNodeManager().getPendingContainerTracker(); + tracker.removePendingAllocation(getNodeManager().getDatanodeInfo(datanodeDetails). + getPendingContainerAllocations(), 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 123189887820..da856681130b 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 @@ -104,12 +104,8 @@ protected void processICR(IncrementalContainerReportFromDatanode report, } if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); - - PendingContainerTracker tracker = - getNodeManager().getPendingContainerTracker(); - if (tracker != null) { - tracker.removePendingAllocation(dd, id); - } + PendingContainerTracker tracker = getNodeManager().getPendingContainerTracker(); + tracker.removePendingAllocation(getNodeManager().getDatanodeInfo(dd).getPendingContainerAllocations(), 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 91cd8cdb0feb..b179871d0839 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 @@ -440,23 +440,4 @@ default void removeNode(DatanodeDetails datanodeDetails) throws NodeNotFoundExce * SCM-side tracker for container allocations not yet reported by datanodes. */ PendingContainerTracker getPendingContainerTracker(); - - /** - * True if the node can accept another container of the given size, accounting for - * {@link #getPendingContainerTracker()}. - */ - boolean hasSpaceForNewContainerAllocation(DatanodeDetails node); - - /** - * Records a pending container allocation for {@code node} so that subsequent - * space checks via {@link #hasSpaceForNewContainerAllocation} account for the - * in-flight allocation before the datanode sends an ICR. - * - * @param node the datanode that will receive the new container replica - * @param containerID the container being allocated - */ - default void recordPendingAllocationForDatanode(DatanodeDetails node, - ContainerID containerID) { - getPendingContainerTracker().recordPendingAllocationForDatanode(node, containerID); - } } 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..934ce06dd15a 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; //TODO /** * 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..a153c167c871 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; @@ -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 f054d9c7d8ac..6ce19566f324 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 @@ -69,7 +69,6 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto.ErrorCode; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; -import org.apache.hadoop.hdds.scm.ScmConfig; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.VersionInfo; import org.apache.hadoop.hdds.scm.container.ContainerID; @@ -145,13 +144,6 @@ public class SCMNodeManager implements NodeManager { private final NonWritableNodeFilter nonWritableNodeFilter; private final int numContainerPerVolume; - /** - * SCM-side pending container allocations per datanode (not yet in container reports). - */ - private final PendingContainerTracker pendingContainerTracker; - - private final long maxContainerSizeBytes; - /** * Lock used to synchronize some operation in Node manager to ensure a * consistent view of the node state. @@ -200,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; @@ -219,14 +214,6 @@ public SCMNodeManager( this.scmContext = scmContext; this.sendCommandNotifyMap = new HashMap<>(); this.nonWritableNodeFilter = new NonWritableNodeFilter(conf); - - this.maxContainerSizeBytes = (long) conf.getStorageSize( - ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, - ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); - ScmConfig scmConfig = conf.getObject(ScmConfig.class); - long rollIntervalMs = scmConfig.getPendingContainerAllocationRollInterval().toMillis(); - this.pendingContainerTracker = new PendingContainerTracker( - maxContainerSizeBytes, rollIntervalMs, this.metrics); } @Override @@ -252,35 +239,6 @@ public PendingContainerTracker getPendingContainerTracker() { return pendingContainerTracker; } - /** - * Effective space check aligned with container allocation: per-disk slot model minus - * SCM pending allocations. - */ - @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node) { - Objects.requireNonNull(node, "node==null"); - try { - DatanodeInfo datanodeInfo = getDatanodeInfo(node); - if (datanodeInfo == null) { - LOG.warn("DatanodeInfo not found for node {}", node.getUuidString()); - return false; - } - return pendingContainerTracker.hasEffectiveAllocatableSpaceForNewContainer( - node, datanodeInfo); - } catch (Exception e) { - LOG.warn("Error checking allocatable space for node {}", node.getUuidString(), e); - return false; - } - } - - @Override - public void recordPendingAllocationForDatanode(DatanodeDetails node, - ContainerID containerID) { - Objects.requireNonNull(node, "node==null"); - Objects.requireNonNull(containerID, "containerID==null"); - pendingContainerTracker.recordPendingAllocationForDatanode(node, containerID); - } - protected NodeStateManager getNodeStateManager() { return nodeStateManager; } @@ -762,7 +720,6 @@ public void processNodeReport(DatanodeDetails datanodeDetails, datanodeInfo.updateStorageReports(nodeReport.getStorageReportList()); datanodeInfo.updateMetaDataStorageReports(nodeReport. getMetadataStorageReportList()); - pendingContainerTracker.rollWindowsIfNeeded(datanodeDetails); metrics.incNumNodeReportProcessed(); } } catch (NodeNotFoundException e) { 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 e4c4646a33ed..739b0c058ec8 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 @@ -232,8 +232,6 @@ void reinitialize(Table pipelineStore) */ boolean hasEnoughSpace(Pipeline pipeline); - void recordPendingAllocation(Pipeline pipeline, ContainerID containerID); - int openContainerLimit(List datanodes); /** 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 b9ac0e874dd8..efc320a1f361 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 @@ -116,6 +116,7 @@ public class MockNodeManager implements NodeManager { private int numPipelinePerDatanode; private PendingContainerTracker pendingContainerTracker; private final OzoneConfiguration conf = new OzoneConfiguration(); + { this.healthyNodes = new LinkedList<>(); this.staleNodes = new LinkedList<>(); @@ -942,6 +943,11 @@ public int openContainerLimit(List datanodes) { return 9; } + @Override + public PendingContainerTracker getPendingContainerTracker() { + return pendingContainerTracker; + } + @Override public long getLastHeartbeat(DatanodeDetails datanodeDetails) { return -1; 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 ae89d696b999..e108d284a77f 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 @@ -455,10 +455,4 @@ public PendingContainerTracker getPendingContainerTracker() { } return pendingContainerTracker; } - - @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node) { - return true; - } - } 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 2889fdcb6cc2..f97279db3104 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 @@ -336,7 +336,7 @@ public boolean isPipelineCreationFrozen() { @Override public boolean hasEnoughSpace(Pipeline pipeline) { for (DatanodeDetails node : pipeline.getNodes()) { - if (!nodeManager.hasSpaceForNewContainerAllocation(node)) { + if (!nodeManager.hasSpaceForNewContainerAllocation(node.getID())) { return false; } } @@ -345,6 +345,9 @@ public boolean hasEnoughSpace(Pipeline pipeline) { @Override public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { + for (DatanodeDetails dn : pipeline.getNodes()) { + nodeManager.recordPendingAllocationForDatanode(dn.getID(), containerID); + } } @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 c305d76e5baa..4e53f6d629ae 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 @@ -76,7 +76,6 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; -import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerInfo; 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 index 83053c5e1f1a..2b84ce901c08 100644 --- 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 @@ -20,17 +20,15 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; -import java.util.ArrayList; -import java.util.List; +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.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; -import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.scm.node.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.SCMNodeManager; import org.apache.hadoop.hdds.scm.node.SCMNodeMetrics; @@ -64,6 +62,7 @@ public class TestPendingContainerTrackerIntegration { private PendingContainerTracker pendingTracker; private SCMNodeMetrics metrics; private OzoneBucket bucket; + private SCMNodeManager nodeManager; @BeforeEach public void setup() throws Exception { @@ -92,7 +91,7 @@ public void setup() throws Exception { // Create bucket for testing bucket = TestDataUtil.createVolumeAndBucket(client); - SCMNodeManager nodeManager = (SCMNodeManager) scm.getScmNodeManager(); + nodeManager = (SCMNodeManager) scm.getScmNodeManager(); assertNotNull(nodeManager); pendingTracker = nodeManager.getPendingContainerTracker(); assertNotNull(pendingTracker, "PendingContainerTracker should be initialized"); @@ -120,39 +119,21 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { long initialRemoved = metrics.getNumPendingContainersRemoved(); // Allocate a container directly - ContainerInfo container = containerManager.allocateContainer( + containerManager.allocateContainer( RatisReplicationConfig.getInstance(ReplicationFactor.THREE), "omServiceIdDefault"); - // Find the container that was allocated - ContainerInfo containerInfo = scm.getContainerManager().getContainers().get(0); - ContainerWithPipeline containerWithPipeline = - scm.getClientProtocolServer().getContainerWithPipeline( - containerInfo.getContainerID()); - - Pipeline pipeline = containerWithPipeline.getPipeline(); - - // Verify pending containers are tracked for all nodes in pipeline - List nodesWithPending = new ArrayList<>(); - for (DatanodeDetails dn : pipeline.getNodes()) { - long pendingCount = pendingTracker.getPendingContainerCount(dn); - if (pendingCount > 0) { - nodesWithPending.add(dn); - LOG.info("DataNode {} has {} pending containers", dn.getUuidString(), pendingCount); - - assertThat(pendingTracker.containsPendingContainer(dn, container.containerID())); - } - } - - assertThat(nodesWithPending).isNotEmpty(); + // Verify the added metric increased, meaning pending was recorded + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersAdded() > initialAdded, + 100, 5000); - // Verify metrics increased long afterAdded = metrics.getNumPendingContainersAdded(); assertThat(afterAdded).isGreaterThan(initialAdded); LOG.info("Pending tracked successfully. Waiting for ICR to remove pending..."); - // Write a key + // Write a key so datanodes send ICRs String keyName = "testKey1"; byte[] data = "Testing Pending Container Tracker".getBytes(UTF_8); @@ -164,34 +145,19 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { } LOG.info("Key written successfully"); - // Wait for ICRs to be sent - GenericTestUtils.waitFor(() -> { - for (DatanodeDetails dn : nodesWithPending) { - if (pendingTracker.containsPendingContainer(dn, container.containerID())) { - LOG.info("Still waiting for ICR from DN {}", dn.getUuidString()); - return false; - } - } + // Wait for ICRs to be processed and removed metric to increase + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersRemoved() > initialRemoved, + 100, 5000); - LOG.info("All pending containers removed via ICR!"); - return true; - }, 100, 5000); - - // Verify all pending containers removed - for (DatanodeDetails dn : nodesWithPending) { - assertFalse(pendingTracker.containsPendingContainer(dn, container.containerID())); - } - - // Verify remove metrics increased long afterRemoved = metrics.getNumPendingContainersRemoved(); assertThat(afterRemoved).isGreaterThan(initialRemoved); - LOG.info("After added = " + afterAdded); - + LOG.info("After added={}, removed={}", afterAdded, afterRemoved); } /** - * Test: Verify idempotency - container reported multiple times. + * Test: Verify idempotency - adding the same container twice does not double-count in metrics. */ @Test public void testIdempotentPendingTracking() throws Exception { @@ -199,24 +165,22 @@ public void testIdempotentPendingTracking() throws Exception { ContainerInfo container = containerManager.allocateContainer( RatisReplicationConfig.getInstance(ReplicationFactor.THREE), "omServiceIdDefault"); - + Pipeline pipeline = scm.getPipelineManager().getPipeline(container.getPipelineID()); DatanodeDetails firstNode = pipeline.getFirstNode(); - - // Record initial state - long initialCount = pendingTracker.getPendingContainerCount(firstNode); + DatanodeInfo firstNodeInfo = nodeManager.getDatanodeInfo(firstNode); - LOG.info("Initial pending state: count={}", initialCount); + // Capture the added metric after initial allocation + long addedAfterAllocation = metrics.getNumPendingContainersAdded(); - // Try adding the same container again (simulates retry or duplicate allocation) - pendingTracker.recordPendingAllocationForDatanode(firstNode, container.containerID()); - - long afterCount = pendingTracker.getPendingContainerCount(firstNode); + LOG.info("Pending added metric after allocation: {}", addedAfterAllocation); - // Count should remain the same (idempotent) - assertEquals(initialCount, afterCount, - "Pending count should not change when adding duplicate container"); + // Try adding the same container again (simulates retry or duplicate allocation) + pendingTracker.recordPendingAllocationForDatanode(firstNodeInfo, container.containerID()); + // The metric should not have increased since it was a duplicate + assertEquals(addedAfterAllocation, metrics.getNumPendingContainersAdded(), + "Pending added metric should not change when adding duplicate container"); } /** @@ -242,15 +206,13 @@ public void testMetricsUpdateThroughLifecycle() throws Exception { } // addedMetrics should increase as containers are allocated - GenericTestUtils.waitFor(() -> { - long afterAdded = metrics.getNumPendingContainersAdded(); - return afterAdded > initialAdded; - }, 100, 5000); - - // Removed metric should increase after icr process - GenericTestUtils.waitFor(() -> { - long afterRemoved = metrics.getNumPendingContainersRemoved(); - return initialRemoved < afterRemoved; - }, 100, 5000); + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersAdded() > initialAdded, + 100, 5000); + + // Removed metric should increase after ICR processing + GenericTestUtils.waitFor( + (BooleanSupplier) () -> metrics.getNumPendingContainersRemoved() > initialRemoved, + 100, 5000); } } From f4a09bfe6fd91676fb638dd9e2ce0df0221997a4 Mon Sep 17 00:00:00 2001 From: ashishk Date: Wed, 29 Apr 2026 14:54:54 +0530 Subject: [PATCH 12/22] Handle datanodeInfo null value --- .../hadoop/hdds/scm/container/ContainerReportHandler.java | 7 +++++-- .../scm/container/IncrementalContainerReportHandler.java | 6 +++++- .../org/apache/hadoop/hdds/scm/node/NodeStateManager.java | 2 +- 3 files changed, 11 insertions(+), 4 deletions(-) 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 dc1f0b586d77..d6f76ef35472 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.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; @@ -179,8 +180,10 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, // Remove from pending tracker when container is added to DN // This container was just confirmed for the first time on this DN PendingContainerTracker tracker = getNodeManager().getPendingContainerTracker(); - tracker.removePendingAllocation(getNodeManager().getDatanodeInfo(datanodeDetails). - getPendingContainerAllocations(), cid); + DatanodeInfo datanodeInfo = getNodeManager().getDatanodeInfo(datanodeDetails); + if (datanodeInfo != null) { + tracker.removePendingAllocation(datanodeInfo.getPendingContainerAllocations(), 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 da856681130b..ae5b2248ec8f 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.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; @@ -105,7 +106,10 @@ protected void processICR(IncrementalContainerReportFromDatanode report, if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); PendingContainerTracker tracker = getNodeManager().getPendingContainerTracker(); - tracker.removePendingAllocation(getNodeManager().getDatanodeInfo(dd).getPendingContainerAllocations(), id); + DatanodeInfo datanodeInfo = getNodeManager().getDatanodeInfo(dd); + if (datanodeInfo != null) { + tracker.removePendingAllocation(datanodeInfo.getPendingContainerAllocations(), id); + } } success = true; } catch (ContainerNotFoundException e) { 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 934ce06dd15a..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 @@ -125,7 +125,7 @@ public class NodeStateManager implements Runnable, Closeable { */ private final long deadNodeIntervalMs; - private final long containerRollIntervalMs; //TODO + private final long containerRollIntervalMs; /** * The future is used to pause/unpause the scheduled checks. From e937485ffe2ac9d035224eabdc9aa68d91ae3316 Mon Sep 17 00:00:00 2001 From: ashishk Date: Fri, 1 May 2026 13:23:48 +0530 Subject: [PATCH 13/22] Make checkEnoughSpace and record atomic --- .../scm/container/ContainerManagerImpl.java | 12 +++--- .../scm/container/ContainerReportHandler.java | 8 +--- .../IncrementalContainerReportHandler.java | 8 +--- .../hadoop/hdds/scm/node/NodeManager.java | 19 +++++++++ .../scm/node/PendingContainerTracker.java | 41 +++++++++++++++++++ .../hadoop/hdds/scm/node/SCMNodeManager.java | 21 ++++++++++ .../hdds/scm/pipeline/PipelineManager.java | 11 +++++ .../scm/pipeline/PipelineManagerImpl.java | 16 ++++++++ .../hdds/scm/container/MockNodeManager.java | 24 +++++++++++ .../scm/container/SimpleMockNodeManager.java | 9 ++++ .../scm/pipeline/MockPipelineManager.java | 10 +++++ 11 files changed, 159 insertions(+), 20 deletions(-) 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 e87bd193b000..3234222f86c7 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 @@ -238,16 +238,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(CONTAINER_ID); 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) @@ -269,7 +270,6 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, } containerStateManager.addContainer(containerInfoBuilder.build()); - pipelineManager.recordPendingAllocation(pipeline, containerID); scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); return containerStateManager.getContainer(containerID); } 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 d6f76ef35472..4053abed19b1 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,9 +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.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode; import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer; @@ -179,11 +177,7 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, 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 - PendingContainerTracker tracker = getNodeManager().getPendingContainerTracker(); - DatanodeInfo datanodeInfo = getNodeManager().getDatanodeInfo(datanodeDetails); - if (datanodeInfo != null) { - tracker.removePendingAllocation(datanodeInfo.getPendingContainerAllocations(), cid); - } + getNodeManager().removePendingAllocationForDatanode(datanodeDetails.getID(), 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 ae5b2248ec8f..7beb7469c4e4 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,9 +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.PendingContainerTracker; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.IncrementalContainerReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventHandler; @@ -105,11 +103,7 @@ protected void processICR(IncrementalContainerReportFromDatanode report, } if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); - PendingContainerTracker tracker = getNodeManager().getPendingContainerTracker(); - DatanodeInfo datanodeInfo = getNodeManager().getDatanodeInfo(dd); - if (datanodeInfo != null) { - tracker.removePendingAllocation(datanodeInfo.getPendingContainerAllocations(), id); - } + getNodeManager().removePendingAllocationForDatanode(dd.getID(), 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 b179871d0839..153ac1bb36ea 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 @@ -197,6 +197,25 @@ default int getAllNodeCount() { */ void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID); + /** + * 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 datanodeID the ID of the DataNode receiving the allocation + * @param containerID the container being allocated + * @return true if space was available and allocation was recorded, false otherwise + */ + boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID); + + /** + * Removes a pending container allocation from a datanode. + * + * @param datanodeID the ID of the DataNode + * @param containerID the container to remove from pending + */ + void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID); + /** * Return the node stat of the specified datanode. * @param datanodeDetails DatanodeDetails. 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 a153c167c871..11d908129480 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 @@ -190,6 +190,47 @@ public boolean hasEffectiveAllocatableSpaceForNewContainer(DatanodeInfo datanode return false; } + /** + * 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 storage reports for the datanode + * @param containerID The container being allocated + * @return true if space was available and allocation was recorded, false otherwise + */ + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { + Objects.requireNonNull(datanodeInfo, "datanodeInfo == null"); + Objects.requireNonNull(containerID, "containerID == null"); + + TwoWindowBucket bucket = datanodeInfo.getPendingContainerAllocations(); + synchronized (bucket) { + long pendingAllocationSize = bucket.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) { + boolean added = bucket.add(containerID); + if (added && metrics != null) { + metrics.incNumPendingContainersAdded(); + } + 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. 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 6ce19566f324..386ff0e04d50 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 @@ -1103,6 +1103,27 @@ public void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerI pendingContainerTracker.recordPendingAllocationForDatanode(datanodeInfo, containerID); } + @Override + public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { + DatanodeInfo datanodeInfo = getNode(datanodeID); + if (datanodeInfo == null) { + LOG.warn("DatanodeInfo not found for node {}", datanodeID); + return false; + } + return pendingContainerTracker.checkSpaceAndRecordAllocation(datanodeInfo, containerID); + } + + @Override + public void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { + DatanodeInfo datanodeInfo = getNode(datanodeID); + if (datanodeInfo == null) { + LOG.warn("DatanodeInfo not found for node {}", datanodeID); + return; + } + pendingContainerTracker.removePendingAllocation( + datanodeInfo.getPendingContainerAllocations(), containerID); + } + /** * Return the node stat of the specified datanode. * 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..326bb6b882ed 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 @@ -232,6 +232,17 @@ void reinitialize(Table pipelineStore) */ boolean hasEnoughSpace(Pipeline pipeline); + /** + * 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 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 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..150e18ba97e7 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; @@ -650,6 +651,21 @@ public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) } } + @Override + public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { + List successfulNodes = new ArrayList<>(); + for (DatanodeDetails dn : pipeline.getNodes()) { + if (!nodeManager.checkSpaceAndRecordAllocation(dn.getID(), containerID)) { + for (DatanodeDetails rollbackNode : successfulNodes) { + nodeManager.removePendingAllocationForDatanode(rollbackNode.getID(), containerID); + } + return false; + } + successfulNodes.add(dn); + } + return true; + } + /** * Schedules a fixed interval job to create pipelines. */ 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 efc320a1f361..65ebcc7b74bd 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 @@ -462,6 +462,30 @@ public void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerI pendingContainerTracker.recordPendingAllocationForDatanode(info, containerID); } + @Override + public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { + 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.checkSpaceAndRecordAllocation(info, containerID); + } + + @Override + public void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { + DatanodeDetails dd = nodeMetricMap.keySet().stream() + .filter(d -> d.getID().equals(datanodeID)) + .findFirst().orElse(null); + DatanodeInfo info = getDatanodeInfo(dd); + if (info != null) { + pendingContainerTracker.removePendingAllocation( + info.getPendingContainerAllocations(), containerID); + } + } + /** * Return the node stat of the specified datanode. * @param datanodeDetails - datanode details. 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 e108d284a77f..9ba73327847b 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 @@ -255,6 +255,15 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dn) { public void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { } + @Override + public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { + return true; + } + + @Override + public void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { + } + @Override public boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID) { return true; 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 f97279db3104..db594e560c57 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 @@ -350,6 +350,16 @@ public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) } } + @Override + public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { + for (DatanodeDetails dn : pipeline.getNodes()) { + if (!nodeManager.checkSpaceAndRecordAllocation(dn.getID(), containerID)) { + return false; + } + } + return true; + } + @Override public int openContainerLimit(List datanodes) { // For tests that do not care about this limit, return a large value. From ac34f96eb4a407df4669253020c1b01b17faa7ec Mon Sep 17 00:00:00 2001 From: ashishk Date: Fri, 1 May 2026 19:38:03 +0530 Subject: [PATCH 14/22] Fix test --- .../scm/container/TestContainerManagerImpl.java | 13 ++++++++----- .../hdds/scm/node/TestContainerPlacement.java | 5 ++++- 2 files changed, 12 insertions(+), 6 deletions(-) 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, From e640c1a6e59ddc5527b83602e03065373e21cb74 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 4 May 2026 10:19:01 +0530 Subject: [PATCH 15/22] Move getDataNodeInfo outside the loop --- .../hdds/scm/container/ContainerReportHandler.java | 4 +++- .../container/IncrementalContainerReportHandler.java | 4 +++- .../org/apache/hadoop/hdds/scm/node/NodeManager.java | 2 +- .../apache/hadoop/hdds/scm/node/SCMNodeManager.java | 7 +------ .../hadoop/hdds/scm/pipeline/PipelineManagerImpl.java | 4 +++- .../hadoop/hdds/scm/container/MockNodeManager.java | 10 +++------- .../hdds/scm/container/SimpleMockNodeManager.java | 2 +- 7 files changed, 15 insertions(+), 18 deletions(-) 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 4053abed19b1..b5c0922cd7f4 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 = getNodeManager().getDatanodeInfo(datanodeDetails); for (ContainerReplicaProto replica : replicas) { ContainerID cid = ContainerID.valueOf(replica.getContainerID()); @@ -177,7 +179,7 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, 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 - getNodeManager().removePendingAllocationForDatanode(datanodeDetails.getID(), cid); + 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 7beb7469c4e4..11dcfd89be10 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 = getNodeManager().getDatanodeInfo(dd); for (ContainerReplicaProto replicaProto : report.getReport().getReportList()) { Object detailsForLogging = getDetailsForLogging(null, replicaProto, dd); @@ -103,7 +105,7 @@ protected void processICR(IncrementalContainerReportFromDatanode report, } if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); - getNodeManager().removePendingAllocationForDatanode(dd.getID(), id); + 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 153ac1bb36ea..76880451a332 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 @@ -214,7 +214,7 @@ default int getAllNodeCount() { * @param datanodeID the ID of the DataNode * @param containerID the container to remove from pending */ - void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID); + void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID); /** * Return the node stat of the specified datanode. 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 386ff0e04d50..7f990ac03158 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 @@ -1114,12 +1114,7 @@ public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID } @Override - public void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { - DatanodeInfo datanodeInfo = getNode(datanodeID); - if (datanodeInfo == null) { - LOG.warn("DatanodeInfo not found for node {}", datanodeID); - return; - } + 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/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index 150e18ba97e7..b820b8d29c67 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 @@ -54,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; @@ -657,7 +658,8 @@ public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID cont for (DatanodeDetails dn : pipeline.getNodes()) { if (!nodeManager.checkSpaceAndRecordAllocation(dn.getID(), containerID)) { for (DatanodeDetails rollbackNode : successfulNodes) { - nodeManager.removePendingAllocationForDatanode(rollbackNode.getID(), containerID); + DatanodeInfo datanodeInfo = nodeManager.getDatanodeInfo(rollbackNode); + nodeManager.removePendingAllocationForDatanode(datanodeInfo, containerID); } return false; } 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 65ebcc7b74bd..b8c1aeeacc2e 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 @@ -475,14 +475,10 @@ public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID } @Override - public void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { - DatanodeDetails dd = nodeMetricMap.keySet().stream() - .filter(d -> d.getID().equals(datanodeID)) - .findFirst().orElse(null); - DatanodeInfo info = getDatanodeInfo(dd); - if (info != null) { + public void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) { + if (datanodeInfo != null) { pendingContainerTracker.removePendingAllocation( - info.getPendingContainerAllocations(), containerID); + datanodeInfo.getPendingContainerAllocations(), containerID); } } 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 9ba73327847b..d4f034abcf4f 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 @@ -261,7 +261,7 @@ public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID } @Override - public void removePendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { + public void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) { } @Override From 863f774d6b24af16b5ab815f1beb18f4897d1199 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 4 May 2026 10:28:01 +0530 Subject: [PATCH 16/22] Add null check --- .../hadoop/hdds/scm/container/ContainerReportHandler.java | 4 +++- .../hdds/scm/container/IncrementalContainerReportHandler.java | 4 +++- .../apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java | 4 +++- 3 files changed, 9 insertions(+), 3 deletions(-) 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 b5c0922cd7f4..9b586a9f4657 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 @@ -179,7 +179,9 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, 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 - getNodeManager().removePendingAllocationForDatanode(datanodeInfo, cid); + 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 11dcfd89be10..38282248f728 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 @@ -105,7 +105,9 @@ protected void processICR(IncrementalContainerReportFromDatanode report, } if (ContainerReportValidator.validate(container, dd, replicaProto)) { processContainerReplica(dd, container, replicaProto, publisher, detailsForLogging); - getNodeManager().removePendingAllocationForDatanode(datanodeInfo, id); + 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/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index b820b8d29c67..e38d7ee7a5a1 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 @@ -659,7 +659,9 @@ public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID cont if (!nodeManager.checkSpaceAndRecordAllocation(dn.getID(), containerID)) { for (DatanodeDetails rollbackNode : successfulNodes) { DatanodeInfo datanodeInfo = nodeManager.getDatanodeInfo(rollbackNode); - nodeManager.removePendingAllocationForDatanode(datanodeInfo, containerID); + if (datanodeInfo != null) { + nodeManager.removePendingAllocationForDatanode(datanodeInfo, containerID); + } } return false; } From c7cc6b96efd10a433b64fdf0c421303c1e09a36e Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 4 May 2026 11:03:07 +0530 Subject: [PATCH 17/22] Update javadoc --- .../main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 76880451a332..6a250d91be84 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 @@ -211,7 +211,7 @@ default int getAllNodeCount() { /** * Removes a pending container allocation from a datanode. * - * @param datanodeID the ID of the DataNode + * @param datanodeInfo info about the datanode * @param containerID the container to remove from pending */ void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID); From 525f9a833edb12f08062b0591760d7f297460f7a Mon Sep 17 00:00:00 2001 From: ashishk Date: Tue, 5 May 2026 10:32:35 +0530 Subject: [PATCH 18/22] Use datanodeinfo directly --- .../hadoop/hdds/scm/container/ContainerReportHandler.java | 2 +- .../hdds/scm/container/IncrementalContainerReportHandler.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 9b586a9f4657..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 @@ -154,7 +154,7 @@ public void onMessage(final ContainerReportFromDatanode reportFromDatanode, containerReport.getReportsList(); final Set expectedContainersInDatanode = getNodeManager().getContainers(datanodeDetails); - DatanodeInfo datanodeInfo = getNodeManager().getDatanodeInfo(datanodeDetails); + DatanodeInfo datanodeInfo = datanodeDetails instanceof DatanodeInfo ? (DatanodeInfo) datanodeDetails : null; for (ContainerReplicaProto replica : replicas) { ContainerID cid = ContainerID.valueOf(replica.getContainerID()); 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 38282248f728..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 @@ -84,7 +84,7 @@ protected void processICR(IncrementalContainerReportFromDatanode report, // issue between the container list in NodeManager and the replicas in // ContainerManager. synchronized (dd) { - DatanodeInfo datanodeInfo = getNodeManager().getDatanodeInfo(dd); + DatanodeInfo datanodeInfo = dd instanceof DatanodeInfo ? (DatanodeInfo) dd : null; for (ContainerReplicaProto replicaProto : report.getReport().getReportList()) { Object detailsForLogging = getDetailsForLogging(null, replicaProto, dd); From 51cadcea51d0e1816ffd41f378d79e201a63ae37 Mon Sep 17 00:00:00 2001 From: ashishk Date: Tue, 5 May 2026 15:33:36 +0530 Subject: [PATCH 19/22] Remove unused methods and move checkSpaceAndAdd to TwoWindowBucket --- .../hadoop/hdds/scm/node/NodeManager.java | 13 -- .../scm/node/PendingContainerTracker.java | 125 ++++++------------ .../hadoop/hdds/scm/node/SCMNodeManager.java | 24 ---- .../hdds/scm/pipeline/PipelineManager.java | 20 --- .../scm/pipeline/PipelineManagerImpl.java | 17 --- .../hdds/scm/container/MockNodeManager.java | 21 --- .../scm/container/SimpleMockNodeManager.java | 9 -- .../scm/node/TestPendingContainerTracker.java | 69 +++++----- .../scm/pipeline/MockPipelineManager.java | 17 --- .../scm/pipeline/TestPipelineManagerImpl.java | 47 ------- ...estPendingContainerTrackerIntegration.java | 31 ----- 11 files changed, 78 insertions(+), 315 deletions(-) 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 6a250d91be84..9eb534793870 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 @@ -184,19 +184,6 @@ default int getAllNodeCount() { @Nullable DatanodeInfo getDatanodeInfo(DatanodeDetails dn); - /** - * True if the node can accept another container of the given size. - */ - boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID); - - /** - * Records a pending container allocation for a single DataNode identified by its ID. - * - * @param datanodeID the ID of the DataNode receiving the allocation - * @param containerID the container being allocated - */ - void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID); - /** * 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 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 11d908129480..61a8b5799426 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 @@ -121,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. */ @@ -149,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) { + long pendingAllocationSize = getCount() * maxContainerSize; + long effectiveAllocatableSpace = 0L; + for (StorageReportProto report : storageReports) { + long usableSpace = VolumeUsage.getUsableSpace(report); + long containersOnThisDisk = usableSpace / maxContainerSize; + effectiveAllocatableSpace += containersOnThisDisk * maxContainerSize; + if (effectiveAllocatableSpace - pendingAllocationSize >= maxContainerSize) { + boolean added = currentWindow.add(containerID); + LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", + containerID, datanodeID, added, getCount()); + return true; + } + } + return false; + } } public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, SCMNodeMetrics metrics) { @@ -159,93 +177,36 @@ 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; - } - - /** - * 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 storage reports for the datanode - * @param containerID The container being allocated - * @return true if space was available and allocation was recorded, false otherwise - */ - public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { - Objects.requireNonNull(datanodeInfo, "datanodeInfo == null"); - Objects.requireNonNull(containerID, "containerID == null"); - TwoWindowBucket bucket = datanodeInfo.getPendingContainerAllocations(); - synchronized (bucket) { - long pendingAllocationSize = bucket.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) { - boolean added = bucket.add(containerID); - if (added && metrics != null) { - metrics.incNumPendingContainersAdded(); - } - return true; - } + boolean added = datanodeInfo.getPendingContainerAllocations() + .checkSpaceAndAdd(storageReports, maxContainerSize, containerID); + if (added) { + if (metrics != null) { + metrics.incNumPendingContainersAdded(); } + } else { 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(); } + return added; } /** 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 7f990ac03158..30a576fc18e7 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 @@ -1079,30 +1079,6 @@ 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); - } - - @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); - } - @Override public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { DatanodeInfo datanodeInfo = getNode(datanodeID); 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 326bb6b882ed..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. * @@ -223,15 +212,6 @@ void reinitialize(Table pipelineStore) */ void releaseWriteLock(); - /** - * Checks whether all Datanodes in the specified pipeline have enough space to store a new container. - * - * @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 - */ - boolean hasEnoughSpace(Pipeline pipeline); - /** * 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 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 e38d7ee7a5a1..834576c96f9e 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 @@ -635,23 +635,6 @@ private boolean isOpenWithUnregisteredNodes(Pipeline pipeline) { return false; } - @Override - public boolean hasEnoughSpace(Pipeline pipeline) { - for (DatanodeDetails node : pipeline.getNodes()) { - if (!nodeManager.hasSpaceForNewContainerAllocation(node.getID())) { - return false; - } - } - return true; - } - - @Override - public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { - for (DatanodeDetails dn : pipeline.getNodes()) { - nodeManager.recordPendingAllocationForDatanode(dn.getID(), containerID); - } - } - @Override public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { List successfulNodes = new ArrayList<>(); 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 b8c1aeeacc2e..6adc1bd2f543 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 @@ -453,15 +453,6 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dd) { return di; } - @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); - } - @Override public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { DatanodeDetails dd = nodeMetricMap.keySet().stream() @@ -981,18 +972,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 d4f034abcf4f..848d04e6cbe7 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 @@ -251,10 +251,6 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dn) { return null; } - @Override - public void recordPendingAllocationForDatanode(DatanodeID datanodeID, ContainerID containerID) { - } - @Override public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { return true; @@ -264,11 +260,6 @@ public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID public void removePendingAllocationForDatanode(DatanodeInfo datanodeInfo, ContainerID containerID) { } - @Override - public boolean hasSpaceForNewContainerAllocation(DatanodeID datanodeID) { - return true; - } - @Override public SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails) { return null; 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 db594e560c57..4c542bd10a19 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 @@ -333,23 +333,6 @@ public boolean isPipelineCreationFrozen() { return false; } - @Override - public boolean hasEnoughSpace(Pipeline pipeline) { - for (DatanodeDetails node : pipeline.getNodes()) { - if (!nodeManager.hasSpaceForNewContainerAllocation(node.getID())) { - return false; - } - } - return true; - } - - @Override - public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { - for (DatanodeDetails dn : pipeline.getNodes()) { - nodeManager.recordPendingAllocationForDatanode(dn.getID(), containerID); - } - } - @Override public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { for (DatanodeDetails dn : pipeline.getNodes()) { 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 index 2b84ce901c08..b3389481bfc1 100644 --- 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 @@ -19,20 +19,16 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertEquals; 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.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; -import org.apache.hadoop.hdds.scm.node.DatanodeInfo; 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.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.TestDataUtil; @@ -156,33 +152,6 @@ public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { LOG.info("After added={}, removed={}", afterAdded, afterRemoved); } - /** - * Test: Verify idempotency - adding the same container twice does not double-count in metrics. - */ - @Test - public void testIdempotentPendingTracking() throws Exception { - // Allocate a container directly - ContainerInfo container = containerManager.allocateContainer( - RatisReplicationConfig.getInstance(ReplicationFactor.THREE), - "omServiceIdDefault"); - - Pipeline pipeline = scm.getPipelineManager().getPipeline(container.getPipelineID()); - DatanodeDetails firstNode = pipeline.getFirstNode(); - DatanodeInfo firstNodeInfo = nodeManager.getDatanodeInfo(firstNode); - - // Capture the added metric after initial allocation - long addedAfterAllocation = metrics.getNumPendingContainersAdded(); - - LOG.info("Pending added metric after allocation: {}", addedAfterAllocation); - - // Try adding the same container again (simulates retry or duplicate allocation) - pendingTracker.recordPendingAllocationForDatanode(firstNodeInfo, container.containerID()); - - // The metric should not have increased since it was a duplicate - assertEquals(addedAfterAllocation, metrics.getNumPendingContainersAdded(), - "Pending added metric should not change when adding duplicate container"); - } - /** * Test: Verify metrics are updated correctly. */ From 1f4e01b7e6b2cf2f744256ccfd143b5222221c23 Mon Sep 17 00:00:00 2001 From: ashishk Date: Tue, 5 May 2026 15:50:10 +0530 Subject: [PATCH 20/22] Fix test --- .../TestPendingContainerTrackerIntegration.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) 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 index b3389481bfc1..13d6bf49fa0e 100644 --- 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 @@ -52,13 +52,10 @@ public class TestPendingContainerTrackerIntegration { private static final Logger LOG = LoggerFactory.getLogger(TestPendingContainerTrackerIntegration.class); private MiniOzoneCluster cluster; - private StorageContainerManager scm; private OzoneClient client; private ContainerManager containerManager; - private PendingContainerTracker pendingTracker; private SCMNodeMetrics metrics; private OzoneBucket bucket; - private SCMNodeManager nodeManager; @BeforeEach public void setup() throws Exception { @@ -80,16 +77,16 @@ public void setup() throws Exception { cluster.waitForClusterToBeReady(); cluster.waitTobeOutOfSafeMode(); - scm = cluster.getStorageContainerManager(); + StorageContainerManager scm = cluster.getStorageContainerManager(); containerManager = scm.getContainerManager(); client = cluster.newClient(); - + // Create bucket for testing bucket = TestDataUtil.createVolumeAndBucket(client); - - nodeManager = (SCMNodeManager) scm.getScmNodeManager(); + + SCMNodeManager nodeManager = (SCMNodeManager) scm.getScmNodeManager(); assertNotNull(nodeManager); - pendingTracker = nodeManager.getPendingContainerTracker(); + PendingContainerTracker pendingTracker = nodeManager.getPendingContainerTracker(); assertNotNull(pendingTracker, "PendingContainerTracker should be initialized"); metrics = pendingTracker.getMetrics(); From 0db2b609bb7a742746cb9ec9d69fdbadd10a7cfc Mon Sep 17 00:00:00 2001 From: ashishk Date: Fri, 29 May 2026 00:35:22 +0530 Subject: [PATCH 21/22] Resolve conflicts --- .../hadoop/hdds/scm/container/ContainerManagerImpl.java | 5 ----- 1 file changed, 5 deletions(-) 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 0fc131ce199b..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,11 +237,6 @@ 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" + From 150528bf95e73edcbde50bc42ab887337f0d57ed Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 1 Jun 2026 13:11:46 +0530 Subject: [PATCH 22/22] Fix review comments --- .../hadoop/hdds/scm/node/NodeManager.java | 4 ++-- .../scm/node/PendingContainerTracker.java | 24 +++++++++---------- .../hadoop/hdds/scm/node/SCMNodeManager.java | 7 +----- .../scm/pipeline/PipelineManagerImpl.java | 24 ++++++++++++------- .../hdds/scm/container/MockNodeManager.java | 10 +++----- .../scm/container/SimpleMockNodeManager.java | 2 +- .../scm/pipeline/MockPipelineManager.java | 2 +- 7 files changed, 35 insertions(+), 38 deletions(-) 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 9eb534793870..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 @@ -189,11 +189,11 @@ default int getAllNodeCount() { * if space is available. This prevents race conditions where multiple threads check space * concurrently and over-allocate. * - * @param datanodeID the ID of the DataNode receiving the allocation + * @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 checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID); + boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID); /** * Removes a pending container allocation from a datanode. 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 61a8b5799426..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 @@ -150,19 +150,19 @@ synchronized int getCount() { * @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) { - long pendingAllocationSize = getCount() * maxContainerSize; - long effectiveAllocatableSpace = 0L; + final int pendingAllocationCount = getCount(); + long allocatableCount = 0; for (StorageReportProto report : storageReports) { - long usableSpace = VolumeUsage.getUsableSpace(report); - long containersOnThisDisk = usableSpace / maxContainerSize; - effectiveAllocatableSpace += containersOnThisDisk * maxContainerSize; - if (effectiveAllocatableSpace - pendingAllocationSize >= maxContainerSize) { - boolean added = currentWindow.add(containerID); + 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 true; + return added; } } return false; @@ -197,12 +197,10 @@ public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, Containe boolean added = datanodeInfo.getPendingContainerAllocations() .checkSpaceAndAdd(storageReports, maxContainerSize, containerID); - if (added) { - if (metrics != null) { + if (metrics != null) { + if (added) { metrics.incNumPendingContainersAdded(); - } - } else { - if (metrics != null) { + } else { metrics.incNumSkippedFullNodeContainerAllocation(); } } 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 30a576fc18e7..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 @@ -1080,12 +1080,7 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dn) { } @Override - public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { - DatanodeInfo datanodeInfo = getNode(datanodeID); - if (datanodeInfo == null) { - LOG.warn("DatanodeInfo not found for node {}", datanodeID); - return false; - } + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { return pendingContainerTracker.checkSpaceAndRecordAllocation(datanodeInfo, containerID); } 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 834576c96f9e..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 @@ -637,14 +637,22 @@ private boolean isOpenWithUnregisteredNodes(Pipeline pipeline) { @Override public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { - List successfulNodes = new ArrayList<>(); - for (DatanodeDetails dn : pipeline.getNodes()) { - if (!nodeManager.checkSpaceAndRecordAllocation(dn.getID(), containerID)) { - for (DatanodeDetails rollbackNode : successfulNodes) { - DatanodeInfo datanodeInfo = nodeManager.getDatanodeInfo(rollbackNode); - if (datanodeInfo != null) { - nodeManager.removePendingAllocationForDatanode(datanodeInfo, 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); + } + + 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; } 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 6adc1bd2f543..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,15 +454,11 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dd) { } @Override - public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { - DatanodeDetails dd = nodeMetricMap.keySet().stream() - .filter(d -> d.getID().equals(datanodeID)) - .findFirst().orElse(null); - DatanodeInfo info = getDatanodeInfo(dd); - if (info == null) { + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { + if (datanodeInfo == null) { return false; } - return pendingContainerTracker.checkSpaceAndRecordAllocation(info, containerID); + return pendingContainerTracker.checkSpaceAndRecordAllocation(datanodeInfo, containerID); } @Override 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 848d04e6cbe7..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 @@ -252,7 +252,7 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dn) { } @Override - public boolean checkSpaceAndRecordAllocation(DatanodeID datanodeID, ContainerID containerID) { + public boolean checkSpaceAndRecordAllocation(DatanodeInfo datanodeInfo, ContainerID containerID) { return true; } 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 4c542bd10a19..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 @@ -336,7 +336,7 @@ public boolean isPipelineCreationFrozen() { @Override public boolean checkSpaceAndRecordAllocation(Pipeline pipeline, ContainerID containerID) { for (DatanodeDetails dn : pipeline.getNodes()) { - if (!nodeManager.checkSpaceAndRecordAllocation(dn.getID(), containerID)) { + if (!nodeManager.checkSpaceAndRecordAllocation(nodeManager.getDatanodeInfo(dn), containerID)) { return false; } }