From 03bc1d270e38eeafa65b24288b40d017a71d7549 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 02:54:02 +0530 Subject: [PATCH 01/14] 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 | 101 ++++- .../scm/container/ContainerReportHandler.java | 9 + .../IncrementalContainerReportHandler.java | 16 + .../container/PendingContainerTracker.java | 338 ++++++++++++++++ .../metrics/SCMContainerManagerMetrics.java | 22 + .../replication/ReplicationManager.java | 38 +- .../hdds/scm/pipeline/PipelineManager.java | 8 + .../scm/pipeline/PipelineManagerImpl.java | 5 + .../TestPendingContainerTracker.java | 375 ++++++++++++++++++ .../scm/pipeline/MockPipelineManager.java | 5 + ...estPendingContainerTrackerIntegration.java | 269 +++++++++++++ 13 files changed, 1214 insertions(+), 4 deletions(-) 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 0876bd22ea4f..ed1c7f68b68f 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 e159eb6948b8..21ea3814a655 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 7780c0839462..5e8d9572f621 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 @@ -34,20 +34,26 @@ import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; +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.container.metrics.SCMContainerManagerMetrics; 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.DatanodeInfo; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException; +import org.apache.hadoop.ozone.container.common.volume.VolumeUsage; import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -81,6 +87,8 @@ public class ContainerManagerImpl implements ContainerManager { private final long maxContainerSize; + private final PendingContainerTracker pendingContainerTracker; + /** * */ @@ -109,7 +117,16 @@ public ContainerManagerImpl( maxContainerSize = (long) conf.getStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); + // Get pending container roll interval from configuration + ScmConfig scmConfig = ((OzoneConfiguration)conf).getObject(ScmConfig.class); + long rollIntervalMs = scmConfig.getPendingContainerAllocationRollInterval().toMillis(); + this.scmContainerManagerMetrics = SCMContainerManagerMetrics.create(); + this.pendingContainerTracker = new PendingContainerTracker( + maxContainerSize, rollIntervalMs, scmContainerManagerMetrics); + + LOG.info("Container allocation pending tracker initialized with maxContainerSize={}B, rollInterval={}ms", + maxContainerSize, rollIntervalMs); } @Override @@ -242,12 +259,75 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner) return containerInfo; } + /** + * Check if a pipeline has sufficient space after considering pending allocations. + * Tracks containers scheduled but not yet written to DataNodes, preventing over-allocation. + * + * @param pipeline The pipeline to check + * @return true if sufficient space is available, false otherwise + */ + private boolean hasSpaceAfterPendingAllocations(Pipeline pipeline) { + try { + for (DatanodeDetails node : pipeline.getNodes()) { + // Get DN's storage statistics + DatanodeInfo datanodeInfo = pipelineManager.getDatanodeInfo(node); + if (datanodeInfo == null) { + LOG.warn("DatanodeInfo not found for node {}", node.getUuidString()); + continue; + } + + List storageReports = datanodeInfo.getStorageReports(); + if (storageReports == null || storageReports.isEmpty()) { + LOG.warn("No storage reports for node {}", node.getUuidString()); + continue; + } + + // Calculate total capacity and effective allocatable space + // For each disk, calculate how many containers can actually fit, + // since containers are written to individual disks, not spread across them. + // Example: disk1=9GB, disk2=14GB with 5GB containers + // (1*5GB) + (2*5GB) = 15GB → actually 3 containers + long totalCapacity = 0L; + long effectiveAllocatableSpace = 0L; + for (StorageReportProto report : storageReports) { + totalCapacity += report.getCapacity(); + long usableSpace = VolumeUsage.getUsableSpace(report); + // Calculate how many containers can fit on this disk + long containersOnThisDisk = usableSpace / maxContainerSize; + // Add effective space (containers that fit * container size) + effectiveAllocatableSpace += containersOnThisDisk * maxContainerSize; + } + + // Get pending allocations from tracker + long pendingAllocations = pendingContainerTracker.getPendingAllocationSize(node); + + // Calculate effective remaining space after pending allocations + long effectiveRemaining = effectiveAllocatableSpace - pendingAllocations; + + // Check if there's enough space for a new container + if (effectiveRemaining < maxContainerSize) { + LOG.info("Node {} insufficient space: capacity={}, effective allocatable={}, " + + "pending allocations={}, effective remaining={}, required={}", + node.getUuidString(), totalCapacity, effectiveAllocatableSpace, + pendingAllocations, effectiveRemaining, maxContainerSize); + return false; + } + } + + return true; + } catch (Exception e) { + LOG.warn("Error checking space for pipeline {}", pipeline.getId(), e); + return true; + } + } + private ContainerInfo allocateContainer(final Pipeline pipeline, final String owner) throws IOException { - if (!pipelineManager.hasEnoughSpace(pipeline, maxContainerSize)) { - LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.", - pipeline, maxContainerSize); + // Check if pipeline has sufficient space after considering recent allocations + if (!hasSpaceAfterPendingAllocations(pipeline)) { + LOG.warn("Pipeline {} does not have sufficient space after considering recent allocations", + pipeline.getId()); return null; } @@ -278,6 +358,11 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, containerStateManager.addContainer(containerInfoBuilder.build()); scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); + // Record pending allocation - tracks containers scheduled but not yet written + pendingContainerTracker.recordPendingAllocation(pipeline, containerID); + LOG.debug("Allocated container {} on pipeline {}. Recorded as pending on {} DataNodes", + containerID, pipeline.getId(), pipeline.getNodes().size()); + return containerStateManager.getContainer(containerID); } @@ -483,4 +568,14 @@ public ContainerStateManager getContainerStateManager() { public SCMHAManager getSCMHAManager() { return haManager; } + + /** + * Get the pending container tracker for tracking scheduled containers. + * Used for removing pending containers when they are confirmed via reports. + * + * @return PendingContainerTracker instance + */ + public PendingContainerTracker getPendingContainerTracker() { + return pendingContainerTracker; + } } 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/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 6a448d6c88df..64a32174c13d 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 @@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerReplica; +import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.utils.db.CodecException; import org.apache.hadoop.hdds.utils.db.RocksDatabaseException; import org.apache.hadoop.hdds.utils.db.Table; @@ -227,4 +228,11 @@ void reinitialize(Table pipelineStore) * Get the pipeline metrics. */ SCMPipelineMetrics getMetrics(); + + /** + * Get DatanodeInfo for a specific DataNode which includes per-volume storage reports. + * @param datanodeDetails The datanode to get info for + * @return DatanodeInfo containing detailed node information including per-disk stats, or null if not available + */ + DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails); } 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 9c529e22e7e1..f13167abfad2 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 @@ -649,6 +649,11 @@ public boolean hasEnoughSpace(Pipeline pipeline, long containerSize) { return true; } + @Override + public DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails) { + return nodeManager.getDatanodeInfo(datanodeDetails); + } + /** * Schedules a fixed interval job to create pipelines. */ 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 d6a3fc546352..32904701e647 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 @@ -346,4 +346,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 0af933cfef5dd037d2dc9bff3cf70384c6b95438 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 03:06:30 +0530 Subject: [PATCH 02/14] 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 a89d153b13740974a4cdc466c28ac7451113892b Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 03:16:31 +0530 Subject: [PATCH 03/14] Safe cast ozone conf --- .../hadoop/hdds/scm/container/ContainerManagerImpl.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 5e8d9572f621..8a83e156a0c2 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 @@ -118,7 +118,10 @@ public ContainerManagerImpl( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); // Get pending container roll interval from configuration - ScmConfig scmConfig = ((OzoneConfiguration)conf).getObject(ScmConfig.class); + OzoneConfiguration ozoneConf = (conf instanceof OzoneConfiguration) + ? (OzoneConfiguration) conf + : new OzoneConfiguration(conf); + ScmConfig scmConfig = ozoneConf.getObject(ScmConfig.class); long rollIntervalMs = scmConfig.getPendingContainerAllocationRollInterval().toMillis(); this.scmContainerManagerMetrics = SCMContainerManagerMetrics.create(); From 82651f5f5348e2c162d20a5af470e6241893f3ac Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 09:53:12 +0530 Subject: [PATCH 04/14] 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 21ea3814a655..e159eb6948b8 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 72cdb6ebb3f9deb335f6ed691c14c447866a5188 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 30 Mar 2026 23:40:02 +0530 Subject: [PATCH 05/14] update test case --- .../container/TestContainerManagerImpl.java | 71 ++++++++++++++----- 1 file changed, 55 insertions(+), 16 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 218a2137e3e6..1f2e0d04845f 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 @@ -27,6 +27,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -43,17 +44,22 @@ 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.DatanodeID; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; +import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps; import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap; import org.apache.hadoop.hdds.scm.ha.SCMHAManager; import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub; import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator; 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.scm.node.NodeStatus; import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; @@ -141,21 +147,42 @@ void testAllocateContainer() throws Exception { */ @Test public void testGetMatchingContainerReturnsNullWhenNotEnoughSpaceInDatanodes() throws IOException { - doReturn(false).when(pipelineManager).hasEnoughSpace(any(), anyLong()); - long sizeRequired = 256 * 1024 * 1024; // 256 MB - Pipeline pipeline = pipelineManager.getPipelines().iterator().next(); - // MockPipelineManager#hasEnoughSpace always returns false - // the pipeline has no existing containers, so a new container gets allocated in getMatchingContainer - ContainerInfo container = containerManager + PipelineManager spyPipelineManager = spy(pipelineManager); + doAnswer(invocation -> { + DatanodeDetails dn = invocation.getArgument(0); + DatanodeInfo info = new DatanodeInfo(dn, + NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), + null); + long gb = 1024L * 1024 * 1024; + // Default SCM max container size is 5GB; 1GB usable => 0 slots => effective remaining < max container size. + StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( + DatanodeID.of(dn.getUuid()), + "/data", + 100L * gb, + 0, + 1 * gb, + HddsProtos.StorageTypeProto.DISK, + false); + info.updateStorageReports(Collections.singletonList(report)); + return info; + }).when(spyPipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); + + File mgrDir = new File(testDir, "matchingContainerNullSpace"); + OzoneConfiguration conf = SCMTestUtils.getConf(mgrDir); + ContainerManager manager = new ContainerManagerImpl(conf, + scmhaManager, sequenceIdGen, spyPipelineManager, + SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); + + Pipeline pipeline = spyPipelineManager.getPipelines().iterator().next(); + ContainerInfo container = manager .getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNull(container); - // create an EC pipeline to test for EC containers ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2); - pipelineManager.createPipeline(ecReplicationConfig); - pipeline = pipelineManager.getPipelines(ecReplicationConfig).iterator().next(); - container = containerManager.getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); + spyPipelineManager.createPipeline(ecReplicationConfig); + pipeline = spyPipelineManager.getPipelines(ecReplicationConfig).iterator().next(); + container = manager.getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNull(container); } @@ -163,12 +190,26 @@ 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 PipelineManager spyPipelineManager = spy(pipelineManager); - doReturn(true).when(spyPipelineManager) - .hasEnoughSpace(any(Pipeline.class), anyLong()); + doAnswer(invocation -> { + DatanodeDetails dn = invocation.getArgument(0); + DatanodeInfo info = new DatanodeInfo(dn, + NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), + null); + long gb = 1024L * 1024 * 1024; + // 50GB usable => multiple 5GB slots under default OZONE_SCM_CONTAINER_SIZE. + StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( + DatanodeID.of(dn.getUuid()), + "/data", + 100L * gb, + 0, + 50L * gb, + HddsProtos.StorageTypeProto.DISK, + false); + info.updateStorageReports(Collections.singletonList(report)); + return info; + }).when(spyPipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); - // create a new ContainerManager using the spy File tempDir = new File(testDir, "tempDir"); OzoneConfiguration conf = SCMTestUtils.getConf(tempDir); ContainerManager manager = new ContainerManagerImpl(conf, @@ -176,12 +217,10 @@ public void testGetMatchingContainerReturnsContainerWhenEnoughSpaceInDatanodes() SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); Pipeline pipeline = spyPipelineManager.getPipelines().iterator().next(); - // the pipeline has no existing containers, so a new container gets allocated in getMatchingContainer ContainerInfo container = manager .getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNotNull(container); - // create an EC pipeline to test for EC containers ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2); spyPipelineManager.createPipeline(ecReplicationConfig); pipeline = spyPipelineManager.getPipelines(ecReplicationConfig).iterator().next(); From fcd46b4ee7b615c8e0787852ddc1fb976b45d6b9 Mon Sep 17 00:00:00 2001 From: ashishk Date: Tue, 31 Mar 2026 00:49:08 +0530 Subject: [PATCH 06/14] Handle unregistered datanodes --- .../hadoop/hdds/scm/container/ContainerManagerImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 8a83e156a0c2..75384134daa1 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 @@ -276,13 +276,13 @@ private boolean hasSpaceAfterPendingAllocations(Pipeline pipeline) { DatanodeInfo datanodeInfo = pipelineManager.getDatanodeInfo(node); if (datanodeInfo == null) { LOG.warn("DatanodeInfo not found for node {}", node.getUuidString()); - continue; + return false; } List storageReports = datanodeInfo.getStorageReports(); if (storageReports == null || storageReports.isEmpty()) { LOG.warn("No storage reports for node {}", node.getUuidString()); - continue; + return false; } // Calculate total capacity and effective allocatable space From 2f6ad9f9a00fbaf8b763b71190bdfa5856a67f80 Mon Sep 17 00:00:00 2001 From: ashishk Date: Tue, 31 Mar 2026 12:36:52 +0530 Subject: [PATCH 07/14] Fix test --- .../container/TestContainerManagerImpl.java | 50 ++++++++----------- .../hdds/scm/node/TestContainerPlacement.java | 24 +++++++-- 2 files changed, 43 insertions(+), 31 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 1f2e0d04845f..fafcd3b05173 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 @@ -26,9 +26,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -109,7 +107,24 @@ 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), anyLong()); + doAnswer(invocation -> { + DatanodeDetails dn = invocation.getArgument(0); + DatanodeInfo info = new DatanodeInfo(dn, + NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), + null); + long gb = 1024L * 1024 * 1024; + // 50GB usable => multiple 5GB slots under default OZONE_SCM_CONTAINER_SIZE. + StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( + DatanodeID.of(dn.getUuid()), + "/data", + 100L * gb, + 0, + 50L * gb, + HddsProtos.StorageTypeProto.DISK, + false); + info.updateStorageReports(Collections.singletonList(report)); + return info; + }).when(pipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); pipelineManager.createPipeline(RatisReplicationConfig.getInstance( ReplicationFactor.THREE)); @@ -189,41 +204,20 @@ public void testGetMatchingContainerReturnsNullWhenNotEnoughSpaceInDatanodes() t @Test public void testGetMatchingContainerReturnsContainerWhenEnoughSpaceInDatanodes() throws IOException { long sizeRequired = 256 * 1024 * 1024; // 256 MB - - PipelineManager spyPipelineManager = spy(pipelineManager); - doAnswer(invocation -> { - DatanodeDetails dn = invocation.getArgument(0); - DatanodeInfo info = new DatanodeInfo(dn, - NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), - null); - long gb = 1024L * 1024 * 1024; - // 50GB usable => multiple 5GB slots under default OZONE_SCM_CONTAINER_SIZE. - StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( - DatanodeID.of(dn.getUuid()), - "/data", - 100L * gb, - 0, - 50L * gb, - HddsProtos.StorageTypeProto.DISK, - false); - info.updateStorageReports(Collections.singletonList(report)); - return info; - }).when(spyPipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); - File tempDir = new File(testDir, "tempDir"); OzoneConfiguration conf = SCMTestUtils.getConf(tempDir); ContainerManager manager = new ContainerManagerImpl(conf, - scmhaManager, sequenceIdGen, spyPipelineManager, + scmhaManager, sequenceIdGen, pipelineManager, SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); - Pipeline pipeline = spyPipelineManager.getPipelines().iterator().next(); + Pipeline pipeline = pipelineManager.getPipelines().iterator().next(); ContainerInfo container = manager .getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNotNull(container); ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2); - spyPipelineManager.createPipeline(ecReplicationConfig); - pipeline = spyPipelineManager.getPipelines(ecReplicationConfig).iterator().next(); + pipelineManager.createPipeline(ecReplicationConfig); + pipeline = pipelineManager.getPipelines(ecReplicationConfig).iterator().next(); container = manager.getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNotNull(container); } 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 9b7c5c77b2cd..e3eb241e8f15 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 @@ -26,8 +26,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -37,6 +36,7 @@ import java.time.Clock; import java.time.ZoneId; import java.util.Arrays; +import java.util.Collections; import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdds.HddsConfigKeys; @@ -160,7 +160,25 @@ SCMNodeManager createNodeManager(OzoneConfiguration config) { ContainerManager createContainerManager() throws IOException { pipelineManager = spy(pipelineManager); - doReturn(true).when(pipelineManager).hasEnoughSpace(any(), anyLong()); + + doAnswer(invocation -> { + DatanodeDetails dn = invocation.getArgument(0); + DatanodeInfo info = new DatanodeInfo(dn, + NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), + null); + long gb = 1024L * 1024 * 1024; + // 50GB usable => multiple 5GB slots under default OZONE_SCM_CONTAINER_SIZE. + StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( + DatanodeID.of(dn.getUuid()), + "/data", + 100L * gb, + 0, + 50L * gb, + HddsProtos.StorageTypeProto.DISK, + false); + info.updateStorageReports(Collections.singletonList(report)); + return info; + }).when(pipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); return new ContainerManagerImpl(conf, scmhaManager, sequenceIdGen, pipelineManager, From 48bbaa3b16d853de18c91b7ed2a9947b59613db7 Mon Sep 17 00:00:00 2001 From: ashishk Date: Sun, 5 Apr 2026 01:36:48 +0530 Subject: [PATCH 08/14] Move PendingContainerTracker to node package and fix review comments --- .../org/apache/hadoop/hdds/scm/ScmConfig.java | 6 +- .../scm/container/ContainerManagerImpl.java | 108 +--- .../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/pipeline/PipelineManagerImpl.java | 7 +- .../scm/server/StorageContainerManager.java | 2 +- .../hdds/scm/block/TestBlockManager.java | 3 +- .../hdds/scm/container/MockNodeManager.java | 28 + .../scm/container/SimpleMockNodeManager.java | 15 + .../container/TestContainerManagerImpl.java | 93 +-- .../TestPendingContainerTracker.java | 375 ------------ .../hdds/scm/node/TestContainerPlacement.java | 30 +- .../scm/node/TestPendingContainerTracker.java | 542 ++++++++---------- .../scm/pipeline/MockPipelineManager.java | 14 +- .../scm/pipeline/TestPipelineManagerImpl.java | 7 + .../scm/safemode/TestSCMSafeModeManager.java | 3 +- ...estPendingContainerTrackerIntegration.java | 20 +- .../recon/scm/ReconContainerManager.java | 6 +- .../ReconStorageContainerManagerFacade.java | 2 +- .../AbstractReconContainerManagerTest.java | 3 +- 27 files changed, 454 insertions(+), 1270 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 ed1c7f68b68f..7c1e48563b67 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 75384134daa1..f8408476dddb 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 @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; +import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.concurrent.locks.Lock; @@ -34,26 +35,22 @@ import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.hdds.client.ECReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; -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.container.metrics.SCMContainerManagerMetrics; 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.DatanodeInfo; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.node.PendingContainerTracker; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException; -import org.apache.hadoop.ozone.container.common.volume.VolumeUsage; import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,7 +83,6 @@ public class ContainerManagerImpl implements ContainerManager { private final Random random = new Random(); private final long maxContainerSize; - private final PendingContainerTracker pendingContainerTracker; /** @@ -98,7 +94,8 @@ public ContainerManagerImpl( final SequenceIdGenerator sequenceIdGen, final PipelineManager pipelineManager, final Table containerStore, - final ContainerReplicaPendingOps containerReplicaPendingOps) + final ContainerReplicaPendingOps containerReplicaPendingOps, + final NodeManager nodeManager) throws IOException { // Introduce builder for this class? this.lock = new ReentrantLock(); @@ -117,19 +114,9 @@ public ContainerManagerImpl( maxContainerSize = (long) conf.getStorageSize(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); - // Get pending container roll interval from configuration - OzoneConfiguration ozoneConf = (conf instanceof OzoneConfiguration) - ? (OzoneConfiguration) conf - : new OzoneConfiguration(conf); - ScmConfig scmConfig = ozoneConf.getObject(ScmConfig.class); - long rollIntervalMs = scmConfig.getPendingContainerAllocationRollInterval().toMillis(); - this.scmContainerManagerMetrics = SCMContainerManagerMetrics.create(); - this.pendingContainerTracker = new PendingContainerTracker( - maxContainerSize, rollIntervalMs, scmContainerManagerMetrics); - - LOG.info("Container allocation pending tracker initialized with maxContainerSize={}B, rollInterval={}ms", - maxContainerSize, rollIntervalMs); + this.pendingContainerTracker = Objects.requireNonNull( + nodeManager.getPendingContainerTracker(), "pendingContainerTracker"); } @Override @@ -262,75 +249,12 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner) return containerInfo; } - /** - * Check if a pipeline has sufficient space after considering pending allocations. - * Tracks containers scheduled but not yet written to DataNodes, preventing over-allocation. - * - * @param pipeline The pipeline to check - * @return true if sufficient space is available, false otherwise - */ - private boolean hasSpaceAfterPendingAllocations(Pipeline pipeline) { - try { - for (DatanodeDetails node : pipeline.getNodes()) { - // Get DN's storage statistics - DatanodeInfo datanodeInfo = pipelineManager.getDatanodeInfo(node); - if (datanodeInfo == null) { - LOG.warn("DatanodeInfo not found for node {}", node.getUuidString()); - return false; - } - - List storageReports = datanodeInfo.getStorageReports(); - if (storageReports == null || storageReports.isEmpty()) { - LOG.warn("No storage reports for node {}", node.getUuidString()); - return false; - } - - // Calculate total capacity and effective allocatable space - // For each disk, calculate how many containers can actually fit, - // since containers are written to individual disks, not spread across them. - // Example: disk1=9GB, disk2=14GB with 5GB containers - // (1*5GB) + (2*5GB) = 15GB → actually 3 containers - long totalCapacity = 0L; - long effectiveAllocatableSpace = 0L; - for (StorageReportProto report : storageReports) { - totalCapacity += report.getCapacity(); - long usableSpace = VolumeUsage.getUsableSpace(report); - // Calculate how many containers can fit on this disk - long containersOnThisDisk = usableSpace / maxContainerSize; - // Add effective space (containers that fit * container size) - effectiveAllocatableSpace += containersOnThisDisk * maxContainerSize; - } - - // Get pending allocations from tracker - long pendingAllocations = pendingContainerTracker.getPendingAllocationSize(node); - - // Calculate effective remaining space after pending allocations - long effectiveRemaining = effectiveAllocatableSpace - pendingAllocations; - - // Check if there's enough space for a new container - if (effectiveRemaining < maxContainerSize) { - LOG.info("Node {} insufficient space: capacity={}, effective allocatable={}, " + - "pending allocations={}, effective remaining={}, required={}", - node.getUuidString(), totalCapacity, effectiveAllocatableSpace, - pendingAllocations, effectiveRemaining, maxContainerSize); - return false; - } - } - - return true; - } catch (Exception e) { - LOG.warn("Error checking space for pipeline {}", pipeline.getId(), e); - return true; - } - } - private ContainerInfo allocateContainer(final Pipeline pipeline, - final String owner) + final String owner) throws IOException { - // Check if pipeline has sufficient space after considering recent allocations - if (!hasSpaceAfterPendingAllocations(pipeline)) { - LOG.warn("Pipeline {} does not have sufficient space after considering recent allocations", - pipeline.getId()); + if (!pipelineManager.hasEnoughSpace(pipeline, maxContainerSize)) { + LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.", + pipeline, maxContainerSize); return null; } @@ -571,14 +495,4 @@ public ContainerStateManager getContainerStateManager() { public SCMHAManager getSCMHAManager() { return haManager; } - - /** - * Get the pending container tracker for tracking scheduled containers. - * Used for removing pending containers when they are confirmed via reports. - * - * @return PendingContainerTracker instance - */ - public PendingContainerTracker getPendingContainerTracker() { - return pendingContainerTracker; - } } 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 e9a019945c1f..29d97100dd0d 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 @@ -422,4 +422,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 3289e7b312a8..e118e6ab3553 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; @@ -143,6 +144,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. @@ -205,6 +213,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 @@ -225,6 +241,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; } @@ -706,6 +751,7 @@ public void processNodeReport(DatanodeDetails datanodeDetails, datanodeInfo.updateStorageReports(nodeReport.getStorageReportList()); datanodeInfo.updateMetaDataStorageReports(nodeReport. getMetadataStorageReportList()); + pendingContainerTracker.rollWindowsIfNeeded(datanodeDetails); metrics.incNumNodeReportProcessed(); } } catch (NodeNotFoundException e) { @@ -1099,6 +1145,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/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index f13167abfad2..e25fee87efc6 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 @@ -44,7 +44,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; -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.ContainerManager; @@ -638,14 +637,10 @@ private boolean isOpenWithUnregisteredNodes(Pipeline pipeline) { @Override public boolean hasEnoughSpace(Pipeline pipeline, long containerSize) { for (DatanodeDetails node : pipeline.getNodes()) { - if (!(node instanceof DatanodeInfo)) { - node = nodeManager.getDatanodeInfo(node); - } - if (!SCMCommonPlacementPolicy.hasEnoughSpace(node, 0, containerSize)) { + if (!nodeManager.hasSpaceForNewContainerAllocation(node, containerSize)) { return false; } } - return true; } 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 013f14b16504..834e85a9747d 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 @@ -61,6 +61,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.Node2PipelineMap; import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; @@ -531,6 +532,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); + } + } } /** @@ -941,6 +949,26 @@ public void setNumHealthyVolumes(int value) { numHealthyDisksPerDatanode = value; } + private PendingContainerTracker pendingContainerTracker; + + @Override + public PendingContainerTracker getPendingContainerTracker() { + if (pendingContainerTracker == null) { + pendingContainerTracker = new PendingContainerTracker(5L * 1024 * 1024 * 1024); + } + return pendingContainerTracker; + } + + @Override + public boolean hasSpaceForNewContainerAllocation(DatanodeDetails node, long containerSize) { + DatanodeInfo info = getDatanodeInfo(node); + if (info == null) { + return false; + } + return getPendingContainerTracker() + .hasEffectiveAllocatableSpaceForNewContainer(node, info, containerSize); + } + /** * 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 4f0679470eab..8ab456d3f2d7 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 @@ -42,6 +42,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; @@ -62,6 +63,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()); @@ -435,4 +437,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/TestContainerManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java index fafcd3b05173..7b12a0fc89ed 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 @@ -26,7 +26,8 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doAnswer; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -42,22 +43,17 @@ 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.DatanodeID; import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; -import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps; import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap; import org.apache.hadoop.hdds.scm.ha.SCMHAManager; import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub; import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator; 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.scm.node.NodeStatus; import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; @@ -88,6 +84,7 @@ public class TestContainerManagerImpl { private SequenceIdGenerator sequenceIdGen; private ContainerReplicaPendingOps pendingOpsMock; private PipelineManager pipelineManager; + private NodeManager nodeManager; @BeforeAll static void init() { @@ -100,31 +97,14 @@ void setUp() throws Exception { final OzoneConfiguration conf = SCMTestUtils.getConf(testDir); dbStore = DBStoreBuilder.createDBStore(conf, SCMDBDefinition.get()); scmhaManager = SCMHAManagerStub.getInstance(true); - NodeManager nodeManager = new MockNodeManager(true, 10); + nodeManager = new MockNodeManager(true, 10); sequenceIdGen = new SequenceIdGenerator( conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore)); PipelineManager base = new MockPipelineManager(dbStore, scmhaManager, nodeManager); pipelineManager = spy(base); // Default: allow allocation in tests unless a test overrides it. - doAnswer(invocation -> { - DatanodeDetails dn = invocation.getArgument(0); - DatanodeInfo info = new DatanodeInfo(dn, - NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), - null); - long gb = 1024L * 1024 * 1024; - // 50GB usable => multiple 5GB slots under default OZONE_SCM_CONTAINER_SIZE. - StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( - DatanodeID.of(dn.getUuid()), - "/data", - 100L * gb, - 0, - 50L * gb, - HddsProtos.StorageTypeProto.DISK, - false); - info.updateStorageReports(Collections.singletonList(report)); - return info; - }).when(pipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); + doReturn(true).when(pipelineManager).hasEnoughSpace(any(Pipeline.class), anyLong()); pipelineManager.createPipeline(RatisReplicationConfig.getInstance( ReplicationFactor.THREE)); @@ -132,7 +112,8 @@ void setUp() throws Exception { pendingOpsMock = mock(ContainerReplicaPendingOps.class); containerManager = new ContainerManagerImpl(conf, scmhaManager, sequenceIdGen, pipelineManager, - SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); + SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock, + nodeManager); } @AfterEach @@ -162,62 +143,50 @@ void testAllocateContainer() throws Exception { */ @Test public void testGetMatchingContainerReturnsNullWhenNotEnoughSpaceInDatanodes() throws IOException { - long sizeRequired = 256 * 1024 * 1024; // 256 MB - PipelineManager spyPipelineManager = spy(pipelineManager); - doAnswer(invocation -> { - DatanodeDetails dn = invocation.getArgument(0); - DatanodeInfo info = new DatanodeInfo(dn, - NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), - null); - long gb = 1024L * 1024 * 1024; - // Default SCM max container size is 5GB; 1GB usable => 0 slots => effective remaining < max container size. - StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( - DatanodeID.of(dn.getUuid()), - "/data", - 100L * gb, - 0, - 1 * gb, - HddsProtos.StorageTypeProto.DISK, - false); - info.updateStorageReports(Collections.singletonList(report)); - return info; - }).when(spyPipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); - - File mgrDir = new File(testDir, "matchingContainerNullSpace"); - OzoneConfiguration conf = SCMTestUtils.getConf(mgrDir); - ContainerManager manager = new ContainerManagerImpl(conf, - scmhaManager, sequenceIdGen, spyPipelineManager, - SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); + doReturn(false).when(pipelineManager).hasEnoughSpace(any(), anyLong()); - Pipeline pipeline = spyPipelineManager.getPipelines().iterator().next(); - ContainerInfo container = manager + long sizeRequired = 256 * 1024 * 1024; // 256 MB + Pipeline pipeline = pipelineManager.getPipelines().iterator().next(); + // MockPipelineManager#hasEnoughSpace 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()); assertNull(container); + // create an EC pipeline to test for EC containers ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2); - spyPipelineManager.createPipeline(ecReplicationConfig); - pipeline = spyPipelineManager.getPipelines(ecReplicationConfig).iterator().next(); - container = manager.getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); + pipelineManager.createPipeline(ecReplicationConfig); + pipeline = pipelineManager.getPipelines(ecReplicationConfig).iterator().next(); + container = containerManager.getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNull(container); } @Test public void testGetMatchingContainerReturnsContainerWhenEnoughSpaceInDatanodes() throws IOException { long sizeRequired = 256 * 1024 * 1024; // 256 MB + + // create a spy to mock hasEnoughSpace to always return true + PipelineManager spyPipelineManager = spy(pipelineManager); + doReturn(true).when(spyPipelineManager) + .hasEnoughSpace(any(Pipeline.class), anyLong()); + + // create a new ContainerManager using the spy File tempDir = new File(testDir, "tempDir"); OzoneConfiguration conf = SCMTestUtils.getConf(tempDir); ContainerManager manager = new ContainerManagerImpl(conf, - scmhaManager, sequenceIdGen, pipelineManager, - SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); + scmhaManager, sequenceIdGen, spyPipelineManager, + SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock, nodeManager); - Pipeline pipeline = pipelineManager.getPipelines().iterator().next(); + Pipeline pipeline = spyPipelineManager.getPipelines().iterator().next(); + // the pipeline has no existing containers, so a new container gets allocated in getMatchingContainer ContainerInfo container = manager .getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNotNull(container); + // create an EC pipeline to test for EC containers ECReplicationConfig ecReplicationConfig = new ECReplicationConfig(3, 2); - pipelineManager.createPipeline(ecReplicationConfig); - pipeline = pipelineManager.getPipelines(ecReplicationConfig).iterator().next(); + spyPipelineManager.createPipeline(ecReplicationConfig); + pipeline = spyPipelineManager.getPipelines(ecReplicationConfig).iterator().next(); container = manager.getMatchingContainer(sizeRequired, "test", pipeline, Collections.emptySet()); assertNotNull(container); } 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 e3eb241e8f15..ea63a81c8fe4 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 @@ -26,7 +26,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doAnswer; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -36,7 +37,6 @@ import java.time.Clock; import java.time.ZoneId; import java.util.Arrays; -import java.util.Collections; import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdds.HddsConfigKeys; @@ -96,6 +96,7 @@ public class TestContainerPlacement { private SequenceIdGenerator sequenceIdGen; private OzoneConfiguration conf; private PipelineManager pipelineManager; + private NodeManager nodeManager; @BeforeEach public void setUp() throws Exception { @@ -105,7 +106,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( @@ -160,31 +161,14 @@ SCMNodeManager createNodeManager(OzoneConfiguration config) { ContainerManager createContainerManager() throws IOException { pipelineManager = spy(pipelineManager); - - doAnswer(invocation -> { - DatanodeDetails dn = invocation.getArgument(0); - DatanodeInfo info = new DatanodeInfo(dn, - NodeStatus.valueOf(HddsProtos.NodeOperationalState.IN_SERVICE, HddsProtos.NodeState.HEALTHY), - null); - long gb = 1024L * 1024 * 1024; - // 50GB usable => multiple 5GB slots under default OZONE_SCM_CONTAINER_SIZE. - StorageContainerDatanodeProtocolProtos.StorageReportProto report = HddsTestUtils.createStorageReport( - DatanodeID.of(dn.getUuid()), - "/data", - 100L * gb, - 0, - 50L * gb, - HddsProtos.StorageTypeProto.DISK, - false); - info.updateStorageReports(Collections.singletonList(report)); - return info; - }).when(pipelineManager).getDatanodeInfo(any(DatanodeDetails.class)); + doReturn(true).when(pipelineManager).hasEnoughSpace(any(), anyLong()); return new ContainerManagerImpl(conf, scmhaManager, sequenceIdGen, pipelineManager, SCMDBDefinition.CONTAINERS.getTable(dbStore), new ContainerReplicaPendingOps( - Clock.system(ZoneId.systemDefault()), null)); + Clock.system(ZoneId.systemDefault()), null), + nodeManager); } /** 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 00a0c565b2ca..0902529f1792 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 @@ -17,371 +17,307 @@ package org.apache.hadoop.hdds.scm.node; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; +import com.google.common.annotations.VisibleForTesting; +import java.util.HashSet; import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.DatanodeID; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; -import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.container.ContainerID; -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; -import org.junit.jupiter.api.Timeout; +import org.apache.hadoop.ozone.container.common.volume.VolumeUsage; +import org.apache.hadoop.util.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * Tests for PendingContainerTracker. + * Tracks per-datanode pending container allocations at SCM using a Two Window Tumbling Bucket + * pattern (similar to 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 5 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:02 | Allocate Container-2      | {C1, C2}      | {}             | {C1, C2}
    + * 00:05 | [ROLL] Window tumbles     | {}            | {C1, C2}       | {C1, C2}
    + * 00:07 | Allocate Container-3      | {C3}          | {C1, C2}       | {C1, C2, C3}
    + * 00:08 | Report confirms C1        | {C3}          | {C2}           | {C2, C3}
    + * 00:10 | [ROLL] Window tumbles     | {}            | {C3}           | {C3}
    + *       | (C2 aged out if not reported)
    + * 
    + * */ -public class TestPendingContainerTracker { - - private static final long MAX_CONTAINER_SIZE = 5L * 1024 * 1024 * 1024; // 5GB - private static final long DEFAULT_ROLL_INTERVAL_MS = 5 * 60 * 1000; - private static final int NUM_DATANODES = 1000; - private static final int NUM_PIPELINES = 1000; - private static final int NUM_CONTAINERS = 10_000; - private List datanodes; - private List pipelines; - private List containers; - - private PendingContainerTracker tracker; - private Pipeline pipeline; - private DatanodeDetails dn1; - private DatanodeDetails dn2; - - /** First three container IDs. */ - private ContainerID container1; - private ContainerID container2; - - @BeforeEach - public void setUp() throws IOException { - tracker = new PendingContainerTracker(MAX_CONTAINER_SIZE, DEFAULT_ROLL_INTERVAL_MS, null); - - datanodes = new ArrayList<>(NUM_DATANODES); - for (int i = 0; i < NUM_DATANODES; i++) { - datanodes.add(MockDatanodeDetails.randomLocalDatanodeDetails()); - } +public class PendingContainerTracker { - pipelines = new ArrayList<>(NUM_PIPELINES); - for (int i = 0; i < NUM_PIPELINES; i++) { - pipelines.add(MockPipeline.createPipeline(Collections.singletonList(datanodes.get(i)))); - } + private static final Logger LOG = LoggerFactory.getLogger(PendingContainerTracker.class); - containers = new ArrayList<>(NUM_CONTAINERS); - for (long id = 1; id <= NUM_CONTAINERS; id++) { - containers.add(ContainerID.valueOf(id)); - } + private final DatanodeBuckets datanodeBuckets; - pipeline = MockPipeline.createPipeline(datanodes.subList(0, 3)); - dn1 = datanodes.get(0); - dn2 = datanodes.get(1); + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; - container1 = containers.get(0); - container2 = containers.get(1); - } + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; - @Test - public void testRecordPendingAllocation() { - // Allocate first 100 containers across first 100 pipelines (1 DN each) - for (int i = 0; i < 100; i++) { - tracker.recordPendingAllocation(pipelines.get(i), containers.get(i)); + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set currentWindow = new HashSet<>(); + private Set previousWindow = new HashSet<>(); + private long lastRollTime = Time.monotonicNow(); + private final long rollIntervalMs; + + TwoWindowBucket(long rollIntervalMs) { + this.rollIntervalMs = rollIntervalMs; } - // Each of the first 100 DNs should have 1 pending container - for (int i = 0; i < 100; i++) { - assertEquals(1, tracker.getPendingContainerCount(datanodes.get(i))); - assertEquals(MAX_CONTAINER_SIZE, - tracker.getPendingContainerCount(datanodes.get(i)) * MAX_CONTAINER_SIZE); + /** + * Roll one or both windows based on elapsed time. + */ + synchronized void rollIfNeeded() { + long now = Time.monotonicNow(); + long elapsed = now - lastRollTime; + + if (elapsed >= 2 * rollIntervalMs) { + int dropped = getCount(); + previousWindow.clear(); + currentWindow.clear(); + lastRollTime = now; + LOG.debug("Double roll interval elapsed ({}ms): dropped {} pending containers", elapsed, dropped); + } else if (elapsed >= rollIntervalMs) { + previousWindow.clear(); + final Set tmp = previousWindow; + previousWindow = currentWindow; + currentWindow = tmp; + lastRollTime = now; + LOG.debug("Rolled window. Previous window size: {} elapsed: ({}ms), Current window reset to empty", + previousWindow.size(), elapsed); + } } - // DNs beyond index 100 should have 0 pending - assertEquals(0, tracker.getPendingContainerCount(datanodes.get(500))); - assertEquals(0, tracker.getPendingContainerCount(datanodes.get(999))); - } - - @Test - public void testRemovePendingAllocation() { - // Allocate containers 0-99 to first 100 pipelines - for (int i = 0; i < 100; i++) { - tracker.recordPendingAllocation(pipelines.get(i), containers.get(i)); + synchronized boolean contains(ContainerID containerID) { + return currentWindow.contains(containerID) || previousWindow.contains(containerID); } - // Remove from first 50 DNs - for (int i = 0; i < 50; i++) { - tracker.removePendingAllocation(datanodes.get(i), containers.get(i)); + /** + * Add container to current window. + */ + synchronized boolean add(ContainerID containerID, DatanodeID dnID) { + boolean added = currentWindow.add(containerID); + LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", + containerID, dnID, added, getCount()); + return added; } - // First 50 DNs should have 0 pending - for (int i = 0; i < 50; i++) { - assertEquals(0, tracker.getPendingContainerCount(datanodes.get(i))); + /** + * Remove container from both windows. + */ + synchronized boolean remove(ContainerID containerID, DatanodeID dnID) { + boolean removedFromCurrent = currentWindow.remove(containerID); + boolean removedFromPrevious = previousWindow.remove(containerID); + boolean removed = removedFromCurrent || removedFromPrevious; + LOG.debug("Removed pending container {} from DataNode {}. Removed={}, Remaining={}", + containerID, dnID, removed, getCount()); + return removed; } - // DNs 50-99 should still have 1 pending - for (int i = 50; i < 100; i++) { - assertEquals(1, tracker.getPendingContainerCount(datanodes.get(i))); + /** + * Count of pending containers in both windows. + */ + synchronized int getCount() { + return currentWindow.size() + previousWindow.size(); } } /** - * After one roll interval, pending entries move from currentWindow to previousWindow and remain - * visible. After a second roll (2× interval total), the old previousWindow is discarded and the - * container ages out if not confirmed. + * Per-datanode two-window buckets. */ - @Test - @Timeout(30) - public void testTwoWindowRollAgesOutContainerAfterTwoIntervals() throws InterruptedException { - long rollMs = 200L; - PendingContainerTracker shortRollTracker = - new PendingContainerTracker(MAX_CONTAINER_SIZE, rollMs, null); - - shortRollTracker.recordPendingAllocationForDatanode(dn1, container1); - assertEquals(1, shortRollTracker.getPendingContainerCount(dn1)); - assertTrue(shortRollTracker.containsPendingContainer(dn1, container1)); - - // First roll: C1 moves from currentWindow to previousWindow; union still includes C1 - Thread.sleep(rollMs + 80); - shortRollTracker.rollWindowsIfNeeded(dn1); - assertEquals(1, shortRollTracker.getPendingContainerCount(dn1)); - assertTrue(shortRollTracker.containsPendingContainer(dn1, container1)); - - // Second roll: prior previousWindow (holding C1) is dropped; C1 is no longer pending - Thread.sleep(rollMs + 80); - shortRollTracker.rollWindowsIfNeeded(dn1); - assertEquals(0, shortRollTracker.getPendingContainerCount(dn1)); - assertEquals(0L, shortRollTracker.getPendingContainerCount(dn1) * MAX_CONTAINER_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.getPendingContainerCount(dn1)); - } - - @Test - public void testUnknownDatanodeHasZeroPendingCount() { - DatanodeDetails unknownDN = MockDatanodeDetails.randomDatanodeDetails(); - assertEquals(0, tracker.getPendingContainerCount(unknownDN)); - } + private static class DatanodeBuckets { + private final ConcurrentHashMap map = new ConcurrentHashMap<>(); + private final long rollIntervalMs; - @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); - } - } - }); + DatanodeBuckets(long rollIntervalMs) { + this.rollIntervalMs = rollIntervalMs; } - // Start all threads - for (Thread thread : threads) { - thread.start(); + TwoWindowBucket get(DatanodeID id) { + final TwoWindowBucket bucket = map.compute(id, (k, b) -> b != null ? b : new TwoWindowBucket(rollIntervalMs)); + bucket.rollIfNeeded(); + return bucket; } - // Wait for all to finish - for (Thread thread : threads) { - thread.join(); + TwoWindowBucket get(DatanodeDetails dn) { + Objects.requireNonNull(dn, "dn == null"); + return get(dn.getID()); } } - @Test - public void testBucketsRetainedWhenEmpty() { - tracker.recordPendingAllocation(pipeline, container1); - - assertEquals(1, tracker.getPendingContainerCount(dn1)); - - // Remove the only pending container from DN1 - tracker.removePendingAllocation(dn1, container1); - - assertEquals(0, tracker.getPendingContainerCount(dn1)); - assertEquals(1, tracker.getPendingContainerCount(dn2)); - - // Empty bucket for DN1 is still usable for new allocations - tracker.recordPendingAllocationForDatanode(dn1, container2); - assertEquals(1, tracker.getPendingContainerCount(dn1)); + public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, + SCMNodeMetrics metrics) { + this.datanodeBuckets = new DatanodeBuckets(rollIntervalMs); + this.maxContainerSize = maxContainerSize; + this.metrics = metrics; + LOG.info("PendingContainerTracker initialized with maxContainerSize={}B, rollInterval={}ms", + maxContainerSize, rollIntervalMs); } - @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.getPendingContainerCount(dn1)); - - // Remove container1 - should work regardless of which window it's in - tracker.removePendingAllocation(dn1, container1); - - assertEquals(1, tracker.getPendingContainerCount(dn1)); - - assertFalse(tracker.containsPendingContainer(dn1, container1)); - assertTrue(tracker.containsPendingContainer(dn1, container2)); + /** + * Advances the two-window tumbling bucket for this datanode when the roll interval has elapsed. + * Call on periodic paths (node report) so windows age even when there are no new + * allocations or container reports touching this tracker. + */ + public void rollWindowsIfNeeded(DatanodeDetails node) { + Objects.requireNonNull(node, "node == null"); + datanodeBuckets.get(node.getID()); } - @Test - public void testManyContainersOnSingleDatanode() { - // Allocate first 1000 containers to the first datanode - DatanodeDetails dn = datanodes.get(0); - for (int i = 0; i < 1000; i++) { - tracker.recordPendingAllocationForDatanode(dn, containers.get(i)); - } - - assertEquals(1000, tracker.getPendingContainerCount(dn)); - assertEquals(1000 * MAX_CONTAINER_SIZE, tracker.getPendingContainerCount(dn) * MAX_CONTAINER_SIZE); - - // Verify specific containers are present - assertTrue(tracker.containsPendingContainer(dn, containers.get(0))); - assertTrue(tracker.containsPendingContainer(dn, containers.get(500))); - assertTrue(tracker.containsPendingContainer(dn, containers.get(999))); - - // Remove half of them - for (int i = 0; i < 500; i++) { - tracker.removePendingAllocation(dn, containers.get(i)); + /** + * 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 {@link #getPendingContainerCount} × {@code maxContainerSize}; + * effective allocatable space sums full-container slots per storage report. + * + * @param node identity used to look up pending allocations (same DN as {@code datanodeInfo}) + * @param datanodeInfo storage reports for the datanode + */ + public boolean hasEffectiveAllocatableSpaceForNewContainer( + DatanodeDetails node, DatanodeInfo datanodeInfo) { + Objects.requireNonNull(node, "node == null"); + Objects.requireNonNull(datanodeInfo, "datanodeInfo == null"); + + long pendingAllocationSize = getPendingContainerCount(node) * maxContainerSize; + List storageReports = datanodeInfo.getStorageReports(); + Objects.requireNonNull(storageReports, "storageReports == null"); + if (storageReports.isEmpty()) { + return false; } - - assertEquals(500, tracker.getPendingContainerCount(dn)); - assertFalse(tracker.containsPendingContainer(dn, containers.get(0))); - assertTrue(tracker.containsPendingContainer(dn, containers.get(999))); - } - - @Test - public void testAllDatanodesWithMultipleContainers() { - // Allocate 10 containers to each of the 1000 datanodes - for (int dnIdx = 0; dnIdx < NUM_DATANODES; dnIdx++) { - DatanodeDetails dn = datanodes.get(dnIdx); - for (int cIdx = 0; cIdx < 10; cIdx++) { - int containerIdx = dnIdx * 10 + cIdx; - tracker.recordPendingAllocationForDatanode(dn, containers.get(containerIdx)); + 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; } } - - // Each DN should have 10 pending containers - for (int i = 0; i < NUM_DATANODES; i++) { - assertEquals(10, tracker.getPendingContainerCount(datanodes.get(i))); - assertEquals(10 * MAX_CONTAINER_SIZE, - tracker.getPendingContainerCount(datanodes.get(i)) * MAX_CONTAINER_SIZE); + if (metrics != null) { + metrics.incNumSkippedFullNodeContainerAllocation(); } + return false; + } - // Remove all containers from every 10th DN - for (int dnIdx = 0; dnIdx < NUM_DATANODES; dnIdx += 10) { - DatanodeDetails dn = datanodes.get(dnIdx); - for (int cIdx = 0; cIdx < 10; cIdx++) { - int containerIdx = dnIdx * 10 + cIdx; - tracker.removePendingAllocation(dn, containers.get(containerIdx)); - } - } + /** + * 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) { + Objects.requireNonNull(pipeline, "pipeline == null"); + Objects.requireNonNull(containerID, "containerID == null"); - // Every 10th DN should have 0 pending - for (int i = 0; i < NUM_DATANODES; i += 10) { - assertEquals(0, tracker.getPendingContainerCount(datanodes.get(i))); + for (DatanodeDetails node : pipeline.getNodes()) { + recordPendingAllocationForDatanode(node, containerID); } - - // Other DNs should still have 10 pending - assertEquals(10, tracker.getPendingContainerCount(datanodes.get(1))); - assertEquals(10, tracker.getPendingContainerCount(datanodes.get(15))); - assertEquals(10, tracker.getPendingContainerCount(datanodes.get(999))); } - @Test - public void testIdempotentRecording() { - // Allocate same 100 containers multiple times to first 100 DNs - DatanodeDetails dn = datanodes.get(0); + /** + * 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) { + Objects.requireNonNull(node, "node == null"); + Objects.requireNonNull(containerID, "containerID == null"); + + DatanodeID dnID = node.getID(); + boolean added = addContainerToBucket(containerID, dnID); - for (int round = 0; round < 5; round++) { - for (int i = 0; i < 100; i++) { - tracker.recordPendingAllocationForDatanode(dn, containers.get(i)); - } + if (added && metrics != null) { + metrics.incNumPendingContainersAdded(); } - - // Should still only have 100 containers - assertEquals(100, tracker.getPendingContainerCount(dn)); } - @Test - public void testMultiVolumeAccumulatedSpaceIsNotEnough() { - DatanodeDetails dn = datanodes.get(0); - long containerSize = MAX_CONTAINER_SIZE; - - List reports = new ArrayList<>(); - reports.add(createStorageReport(dn, 100 * containerSize, containerSize / 4, 0)); - reports.add(createStorageReport(dn, 100 * containerSize, containerSize / 4, 0)); - reports.add(createStorageReport(dn, 100 * containerSize, containerSize / 2, 0)); - DatanodeInfo dnInfo = new DatanodeInfo(dn, NodeStatus.inServiceHealthy(), null); - dnInfo.updateStorageReports(reports); - - assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); + private boolean addContainerToBucket(ContainerID containerID, DatanodeID dnID) { + return datanodeBuckets.get(dnID).add(containerID, dnID); } - @Test - public void testMultiVolumeWithPendingAllocation() { - DatanodeDetails dn = datanodes.get(0); - long containerSize = MAX_CONTAINER_SIZE; + /** + * 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) { + Objects.requireNonNull(node, "node == null"); + Objects.requireNonNull(containerID, "containerID == null"); - // Remaining space available for 3 containers across all the volumes - tracker.recordPendingAllocationForDatanode(dn, containers.get(0)); - tracker.recordPendingAllocationForDatanode(dn, containers.get(1)); + DatanodeID dnID = node.getID(); + boolean removed = removeContainerFromBucket(containerID, dnID); - List reports = new ArrayList<>(); - reports.add(createStorageReport(dn, 100 * containerSize, containerSize, 0)); - reports.add(createStorageReport(dn, 50 * containerSize, containerSize, 0)); - reports.add(createStorageReport(dn, 100 * containerSize, containerSize, 0)); - DatanodeInfo dnInfo = new DatanodeInfo(dn, NodeStatus.inServiceHealthy(), null); - dnInfo.updateStorageReports(reports); - // Remaining space available for 1 container across all the volume after 2 container allocation + if (removed && metrics != null) { + metrics.incNumPendingContainersRemoved(); + } + } - assertTrue(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); + private boolean removeContainerFromBucket(ContainerID containerID, DatanodeID dnID) { + return datanodeBuckets.get(dnID).remove(containerID, dnID); + } - tracker.recordPendingAllocationForDatanode(dn, containers.get(2)); - // Remaining space available for 0 container across all the volume - assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); + /** + * Number of pending container allocations for this datanode (union of current and previous + * windows). This call may advance the internal tumbling window if the roll interval has elapsed. + * + * @param node The DataNode + * @return Pending container count + */ + public long getPendingContainerCount(DatanodeDetails node) { + Objects.requireNonNull(node, "node == null"); + return datanodeBuckets.get(node).getCount(); } - @Test - public void testMultiVolumeWithCommittedBytes() { - DatanodeDetails dn = datanodes.get(0); - long containerSize = MAX_CONTAINER_SIZE; - - List reports = new ArrayList<>(); - reports.add(createStorageReport(dn, 100 * containerSize, 6 * containerSize, 5 * containerSize)); - reports.add(createStorageReport(dn, 50 * containerSize, 3 * containerSize, 3 * containerSize)); - DatanodeInfo dnInfo = new DatanodeInfo(dn, NodeStatus.inServiceHealthy(), null); - dnInfo.updateStorageReports(reports); - // Remaining space available for 1 container across all the volume considering committed bytes - - assertTrue(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); - tracker.recordPendingAllocationForDatanode(dn, containers.get(0)); - // Remaining space available for 0 container across all the volume considering - // committed bytes and container allocation - assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); + /** + * Whether container is in the current or previous window for this datanode. + */ + @VisibleForTesting + public boolean containsPendingContainer(DatanodeDetails node, ContainerID containerID) { + Objects.requireNonNull(node, "node == null"); + Objects.requireNonNull(containerID, "containerID == null"); + return datanodeBuckets.get(node).contains(containerID); } - private StorageReportProto createStorageReport(DatanodeDetails dn, long capacity, long remaining, long committed) { - return HddsTestUtils.createStorageReports(dn.getID(), capacity, remaining, committed).get(0); + @VisibleForTesting + public SCMNodeMetrics getMetrics() { + return metrics; } } 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 32904701e647..4086c9484729 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; @@ -50,9 +51,11 @@ public class MockPipelineManager implements PipelineManager { private final PipelineStateManager stateManager; + private final NodeManager nodeManager; public MockPipelineManager(DBStore dbStore, SCMHAManager scmhaManager, NodeManager nodeManager) throws RocksDatabaseException, CodecException, DuplicatedPipelineIdException { + this.nodeManager = nodeManager; stateManager = PipelineStateManagerImpl .newBuilder().setNodeManager(nodeManager) .setRatisServer(scmhaManager.getRatisServer()) @@ -333,7 +336,12 @@ public boolean isPipelineCreationFrozen() { @Override public boolean hasEnoughSpace(Pipeline pipeline, long containerSize) { - return false; + for (DatanodeDetails node : pipeline.getNodes()) { + if (!nodeManager.hasSpaceForNewContainerAllocation(node, containerSize)) { + return false; + } + } + return true; } @Override @@ -348,7 +356,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 e7fc6f14f9b6..d7a0e682fab4 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; @@ -976,6 +977,12 @@ public void testHasEnoughSpace() throws IOException { doReturn(info).when(mockedNodeManager).getDatanodeInfo(dn); datanodeInfoList.add(info); } + doAnswer(invocation -> { + DatanodeDetails dn = invocation.getArgument(0); + long cs = invocation.getArgument(1); + DatanodeInfo info = mockedNodeManager.getDatanodeInfo(dn); + return SCMCommonPlacementPolicy.hasEnoughSpace(info, 0, cs); + }).when(mockedNodeManager).hasSpaceForNewContainerAllocation(any(DatanodeDetails.class), anyLong()); assertTrue(pipelineManager.hasEnoughSpace(pipeline, containerSize)); // Case 2: One node does not have enough space. 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 8542a70d41e78dbf97dacd584784343dc4101056 Mon Sep 17 00:00:00 2001 From: ashishk Date: Sun, 5 Apr 2026 01:52:31 +0530 Subject: [PATCH 09/14] 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 | 3 +-- 3 files changed, 3 insertions(+), 4 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 7c1e48563b67..1e7aece7ed09 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 f8408476dddb..a265a810c1a5 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 @@ -250,7 +250,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, maxContainerSize)) { LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.", 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 834e85a9747d..c7bffa7e1b1a 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 @@ -114,6 +114,7 @@ public class MockNodeManager implements NodeManager { private ConcurrentMap> dnsToUuidMap; private int numHealthyDisksPerDatanode; private int numPipelinePerDatanode; + private PendingContainerTracker pendingContainerTracker; { this.healthyNodes = new LinkedList<>(); @@ -949,8 +950,6 @@ public void setNumHealthyVolumes(int value) { numHealthyDisksPerDatanode = value; } - private PendingContainerTracker pendingContainerTracker; - @Override public PendingContainerTracker getPendingContainerTracker() { if (pendingContainerTracker == null) { From c2fcb1f94d42be0430184fde369a65684694235c Mon Sep 17 00:00:00 2001 From: ashishk Date: Tue, 7 Apr 2026 12:19:28 +0530 Subject: [PATCH 10/14] 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 c7bffa7e1b1a..8e94a384a0bd 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 @@ -533,13 +533,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 79d76b1383172a36a774bf542bbbb86e8ce5f92d Mon Sep 17 00:00:00 2001 From: ashishk Date: Wed, 8 Apr 2026 13:41:56 +0530 Subject: [PATCH 11/14] Fix review comments --- .../hdds/scm/pipeline/PipelineManager.java | 8 - .../scm/pipeline/PipelineManagerImpl.java | 6 - .../scm/node/TestPendingContainerTracker.java | 542 ++++++++++-------- .../scm/pipeline/MockPipelineManager.java | 6 - ...estPendingContainerTrackerIntegration.java | 10 +- 5 files changed, 308 insertions(+), 264 deletions(-) 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 64a32174c13d..6a448d6c88df 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 @@ -27,7 +27,6 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.node.DatanodeInfo; import org.apache.hadoop.hdds.utils.db.CodecException; import org.apache.hadoop.hdds.utils.db.RocksDatabaseException; import org.apache.hadoop.hdds.utils.db.Table; @@ -228,11 +227,4 @@ void reinitialize(Table pipelineStore) * Get the pipeline metrics. */ SCMPipelineMetrics getMetrics(); - - /** - * Get DatanodeInfo for a specific DataNode which includes per-volume storage reports. - * @param datanodeDetails The datanode to get info for - * @return DatanodeInfo containing detailed node information including per-disk stats, or null if not available - */ - DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails); } 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 e25fee87efc6..b1de6e9569e7 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 @@ -53,7 +53,6 @@ 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; @@ -644,11 +643,6 @@ public boolean hasEnoughSpace(Pipeline pipeline, long containerSize) { return true; } - @Override - public DatanodeInfo getDatanodeInfo(DatanodeDetails datanodeDetails) { - return nodeManager.getDatanodeInfo(datanodeDetails); - } - /** * Schedules a fixed interval job to create pipelines. */ 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 0902529f1792..00a0c565b2ca 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 @@ -17,307 +17,371 @@ package org.apache.hadoop.hdds.scm.node; -import com.google.common.annotations.VisibleForTesting; -import java.util.HashSet; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; 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.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; -import org.apache.hadoop.ozone.container.common.volume.VolumeUsage; -import org.apache.hadoop.util.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** - * Tracks per-datanode pending container allocations at SCM using a Two Window Tumbling Bucket - * pattern (similar to 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 5 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:02 | Allocate Container-2      | {C1, C2}      | {}             | {C1, C2}
    - * 00:05 | [ROLL] Window tumbles     | {}            | {C1, C2}       | {C1, C2}
    - * 00:07 | Allocate Container-3      | {C3}          | {C1, C2}       | {C1, C2, C3}
    - * 00:08 | Report confirms C1        | {C3}          | {C2}           | {C2, C3}
    - * 00:10 | [ROLL] Window tumbles     | {}            | {C3}           | {C3}
    - *       | (C2 aged out if not reported)
    - * 
    - * + * Tests for PendingContainerTracker. */ -public class PendingContainerTracker { +public class TestPendingContainerTracker { + + private static final long MAX_CONTAINER_SIZE = 5L * 1024 * 1024 * 1024; // 5GB + private static final long DEFAULT_ROLL_INTERVAL_MS = 5 * 60 * 1000; + private static final int NUM_DATANODES = 1000; + private static final int NUM_PIPELINES = 1000; + private static final int NUM_CONTAINERS = 10_000; + private List datanodes; + private List pipelines; + private List containers; + + private PendingContainerTracker tracker; + private Pipeline pipeline; + private DatanodeDetails dn1; + private DatanodeDetails dn2; + + /** First three container IDs. */ + private ContainerID container1; + private ContainerID container2; + + @BeforeEach + public void setUp() throws IOException { + tracker = new PendingContainerTracker(MAX_CONTAINER_SIZE, DEFAULT_ROLL_INTERVAL_MS, null); + + datanodes = new ArrayList<>(NUM_DATANODES); + for (int i = 0; i < NUM_DATANODES; i++) { + datanodes.add(MockDatanodeDetails.randomLocalDatanodeDetails()); + } - private static final Logger LOG = LoggerFactory.getLogger(PendingContainerTracker.class); + pipelines = new ArrayList<>(NUM_PIPELINES); + for (int i = 0; i < NUM_PIPELINES; i++) { + pipelines.add(MockPipeline.createPipeline(Collections.singletonList(datanodes.get(i)))); + } - private final DatanodeBuckets datanodeBuckets; + containers = new ArrayList<>(NUM_CONTAINERS); + for (long id = 1; id <= NUM_CONTAINERS; id++) { + containers.add(ContainerID.valueOf(id)); + } - /** - * Maximum container size in bytes. - */ - private final long maxContainerSize; + pipeline = MockPipeline.createPipeline(datanodes.subList(0, 3)); + dn1 = datanodes.get(0); + dn2 = datanodes.get(1); - /** - * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). - */ - private final SCMNodeMetrics metrics; + container1 = containers.get(0); + container2 = containers.get(1); + } - /** - * Two-window bucket for a single DataNode. - * Contains current and previous window sets, plus last roll timestamp. - */ - private static class TwoWindowBucket { - private Set currentWindow = new HashSet<>(); - private Set previousWindow = new HashSet<>(); - private long lastRollTime = Time.monotonicNow(); - private final long rollIntervalMs; - - TwoWindowBucket(long rollIntervalMs) { - this.rollIntervalMs = rollIntervalMs; + @Test + public void testRecordPendingAllocation() { + // Allocate first 100 containers across first 100 pipelines (1 DN each) + for (int i = 0; i < 100; i++) { + tracker.recordPendingAllocation(pipelines.get(i), containers.get(i)); } - /** - * Roll one or both windows based on elapsed time. - */ - synchronized void rollIfNeeded() { - long now = Time.monotonicNow(); - long elapsed = now - lastRollTime; - - if (elapsed >= 2 * rollIntervalMs) { - int dropped = getCount(); - previousWindow.clear(); - currentWindow.clear(); - lastRollTime = now; - LOG.debug("Double roll interval elapsed ({}ms): dropped {} pending containers", elapsed, dropped); - } else if (elapsed >= rollIntervalMs) { - previousWindow.clear(); - final Set tmp = previousWindow; - previousWindow = currentWindow; - currentWindow = tmp; - lastRollTime = now; - LOG.debug("Rolled window. Previous window size: {} elapsed: ({}ms), Current window reset to empty", - previousWindow.size(), elapsed); - } + // Each of the first 100 DNs should have 1 pending container + for (int i = 0; i < 100; i++) { + assertEquals(1, tracker.getPendingContainerCount(datanodes.get(i))); + assertEquals(MAX_CONTAINER_SIZE, + tracker.getPendingContainerCount(datanodes.get(i)) * MAX_CONTAINER_SIZE); } - synchronized boolean contains(ContainerID containerID) { - return currentWindow.contains(containerID) || previousWindow.contains(containerID); + // DNs beyond index 100 should have 0 pending + assertEquals(0, tracker.getPendingContainerCount(datanodes.get(500))); + assertEquals(0, tracker.getPendingContainerCount(datanodes.get(999))); + } + + @Test + public void testRemovePendingAllocation() { + // Allocate containers 0-99 to first 100 pipelines + for (int i = 0; i < 100; i++) { + tracker.recordPendingAllocation(pipelines.get(i), containers.get(i)); } - /** - * Add container to current window. - */ - synchronized boolean add(ContainerID containerID, DatanodeID dnID) { - boolean added = currentWindow.add(containerID); - LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", - containerID, dnID, added, getCount()); - return added; + // Remove from first 50 DNs + for (int i = 0; i < 50; i++) { + tracker.removePendingAllocation(datanodes.get(i), containers.get(i)); } - /** - * Remove container from both windows. - */ - synchronized boolean remove(ContainerID containerID, DatanodeID dnID) { - boolean removedFromCurrent = currentWindow.remove(containerID); - boolean removedFromPrevious = previousWindow.remove(containerID); - boolean removed = removedFromCurrent || removedFromPrevious; - LOG.debug("Removed pending container {} from DataNode {}. Removed={}, Remaining={}", - containerID, dnID, removed, getCount()); - return removed; + // First 50 DNs should have 0 pending + for (int i = 0; i < 50; i++) { + assertEquals(0, tracker.getPendingContainerCount(datanodes.get(i))); } - /** - * Count of pending containers in both windows. - */ - synchronized int getCount() { - return currentWindow.size() + previousWindow.size(); + // DNs 50-99 should still have 1 pending + for (int i = 50; i < 100; i++) { + assertEquals(1, tracker.getPendingContainerCount(datanodes.get(i))); } } /** - * Per-datanode two-window buckets. + * After one roll interval, pending entries move from currentWindow to previousWindow and remain + * visible. After a second roll (2× interval total), the old previousWindow is discarded and the + * container ages out if not confirmed. */ - private static class DatanodeBuckets { - private final ConcurrentHashMap map = new ConcurrentHashMap<>(); - private final long rollIntervalMs; + @Test + @Timeout(30) + public void testTwoWindowRollAgesOutContainerAfterTwoIntervals() throws InterruptedException { + long rollMs = 200L; + PendingContainerTracker shortRollTracker = + new PendingContainerTracker(MAX_CONTAINER_SIZE, rollMs, null); + + shortRollTracker.recordPendingAllocationForDatanode(dn1, container1); + assertEquals(1, shortRollTracker.getPendingContainerCount(dn1)); + assertTrue(shortRollTracker.containsPendingContainer(dn1, container1)); + + // First roll: C1 moves from currentWindow to previousWindow; union still includes C1 + Thread.sleep(rollMs + 80); + shortRollTracker.rollWindowsIfNeeded(dn1); + assertEquals(1, shortRollTracker.getPendingContainerCount(dn1)); + assertTrue(shortRollTracker.containsPendingContainer(dn1, container1)); + + // Second roll: prior previousWindow (holding C1) is dropped; C1 is no longer pending + Thread.sleep(rollMs + 80); + shortRollTracker.rollWindowsIfNeeded(dn1); + assertEquals(0, shortRollTracker.getPendingContainerCount(dn1)); + assertEquals(0L, shortRollTracker.getPendingContainerCount(dn1) * MAX_CONTAINER_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.getPendingContainerCount(dn1)); + } + + @Test + public void testUnknownDatanodeHasZeroPendingCount() { + DatanodeDetails unknownDN = MockDatanodeDetails.randomDatanodeDetails(); + assertEquals(0, tracker.getPendingContainerCount(unknownDN)); + } - DatanodeBuckets(long rollIntervalMs) { - this.rollIntervalMs = rollIntervalMs; + @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); + } + } + }); } - TwoWindowBucket get(DatanodeID id) { - final TwoWindowBucket bucket = map.compute(id, (k, b) -> b != null ? b : new TwoWindowBucket(rollIntervalMs)); - bucket.rollIfNeeded(); - return bucket; + // Start all threads + for (Thread thread : threads) { + thread.start(); } - TwoWindowBucket get(DatanodeDetails dn) { - Objects.requireNonNull(dn, "dn == null"); - return get(dn.getID()); + // Wait for all to finish + for (Thread thread : threads) { + thread.join(); } } - public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, - SCMNodeMetrics metrics) { - this.datanodeBuckets = new DatanodeBuckets(rollIntervalMs); - this.maxContainerSize = maxContainerSize; - this.metrics = metrics; - LOG.info("PendingContainerTracker initialized with maxContainerSize={}B, rollInterval={}ms", - maxContainerSize, rollIntervalMs); + @Test + public void testBucketsRetainedWhenEmpty() { + tracker.recordPendingAllocation(pipeline, container1); + + assertEquals(1, tracker.getPendingContainerCount(dn1)); + + // Remove the only pending container from DN1 + tracker.removePendingAllocation(dn1, container1); + + assertEquals(0, tracker.getPendingContainerCount(dn1)); + assertEquals(1, tracker.getPendingContainerCount(dn2)); + + // Empty bucket for DN1 is still usable for new allocations + tracker.recordPendingAllocationForDatanode(dn1, container2); + assertEquals(1, tracker.getPendingContainerCount(dn1)); } - /** - * Advances the two-window tumbling bucket for this datanode when the roll interval has elapsed. - * Call on periodic paths (node report) so windows age even when there are no new - * allocations or container reports touching this tracker. - */ - public void rollWindowsIfNeeded(DatanodeDetails node) { - Objects.requireNonNull(node, "node == null"); - datanodeBuckets.get(node.getID()); + @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.getPendingContainerCount(dn1)); + + // Remove container1 - should work regardless of which window it's in + tracker.removePendingAllocation(dn1, container1); + + assertEquals(1, tracker.getPendingContainerCount(dn1)); + + assertFalse(tracker.containsPendingContainer(dn1, container1)); + assertTrue(tracker.containsPendingContainer(dn1, container2)); } - /** - * 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 {@link #getPendingContainerCount} × {@code maxContainerSize}; - * effective allocatable space sums full-container slots per storage report. - * - * @param node identity used to look up pending allocations (same DN as {@code datanodeInfo}) - * @param datanodeInfo storage reports for the datanode - */ - public boolean hasEffectiveAllocatableSpaceForNewContainer( - DatanodeDetails node, DatanodeInfo datanodeInfo) { - Objects.requireNonNull(node, "node == null"); - Objects.requireNonNull(datanodeInfo, "datanodeInfo == null"); - - long pendingAllocationSize = getPendingContainerCount(node) * maxContainerSize; - List storageReports = datanodeInfo.getStorageReports(); - Objects.requireNonNull(storageReports, "storageReports == null"); - if (storageReports.isEmpty()) { - return false; + @Test + public void testManyContainersOnSingleDatanode() { + // Allocate first 1000 containers to the first datanode + DatanodeDetails dn = datanodes.get(0); + for (int i = 0; i < 1000; i++) { + tracker.recordPendingAllocationForDatanode(dn, containers.get(i)); + } + + assertEquals(1000, tracker.getPendingContainerCount(dn)); + assertEquals(1000 * MAX_CONTAINER_SIZE, tracker.getPendingContainerCount(dn) * MAX_CONTAINER_SIZE); + + // Verify specific containers are present + assertTrue(tracker.containsPendingContainer(dn, containers.get(0))); + assertTrue(tracker.containsPendingContainer(dn, containers.get(500))); + assertTrue(tracker.containsPendingContainer(dn, containers.get(999))); + + // Remove half of them + for (int i = 0; i < 500; i++) { + tracker.removePendingAllocation(dn, containers.get(i)); } - 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; + + assertEquals(500, tracker.getPendingContainerCount(dn)); + assertFalse(tracker.containsPendingContainer(dn, containers.get(0))); + assertTrue(tracker.containsPendingContainer(dn, containers.get(999))); + } + + @Test + public void testAllDatanodesWithMultipleContainers() { + // Allocate 10 containers to each of the 1000 datanodes + for (int dnIdx = 0; dnIdx < NUM_DATANODES; dnIdx++) { + DatanodeDetails dn = datanodes.get(dnIdx); + for (int cIdx = 0; cIdx < 10; cIdx++) { + int containerIdx = dnIdx * 10 + cIdx; + tracker.recordPendingAllocationForDatanode(dn, containers.get(containerIdx)); } } - if (metrics != null) { - metrics.incNumSkippedFullNodeContainerAllocation(); + + // Each DN should have 10 pending containers + for (int i = 0; i < NUM_DATANODES; i++) { + assertEquals(10, tracker.getPendingContainerCount(datanodes.get(i))); + assertEquals(10 * MAX_CONTAINER_SIZE, + tracker.getPendingContainerCount(datanodes.get(i)) * MAX_CONTAINER_SIZE); } - return false; - } - /** - * 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) { - Objects.requireNonNull(pipeline, "pipeline == null"); - Objects.requireNonNull(containerID, "containerID == null"); + // Remove all containers from every 10th DN + for (int dnIdx = 0; dnIdx < NUM_DATANODES; dnIdx += 10) { + DatanodeDetails dn = datanodes.get(dnIdx); + for (int cIdx = 0; cIdx < 10; cIdx++) { + int containerIdx = dnIdx * 10 + cIdx; + tracker.removePendingAllocation(dn, containers.get(containerIdx)); + } + } - for (DatanodeDetails node : pipeline.getNodes()) { - recordPendingAllocationForDatanode(node, containerID); + // Every 10th DN should have 0 pending + for (int i = 0; i < NUM_DATANODES; i += 10) { + assertEquals(0, tracker.getPendingContainerCount(datanodes.get(i))); } - } - /** - * 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) { - Objects.requireNonNull(node, "node == null"); - Objects.requireNonNull(containerID, "containerID == null"); + // Other DNs should still have 10 pending + assertEquals(10, tracker.getPendingContainerCount(datanodes.get(1))); + assertEquals(10, tracker.getPendingContainerCount(datanodes.get(15))); + assertEquals(10, tracker.getPendingContainerCount(datanodes.get(999))); + } - DatanodeID dnID = node.getID(); - boolean added = addContainerToBucket(containerID, dnID); + @Test + public void testIdempotentRecording() { + // Allocate same 100 containers multiple times to first 100 DNs + DatanodeDetails dn = datanodes.get(0); - if (added && metrics != null) { - metrics.incNumPendingContainersAdded(); + for (int round = 0; round < 5; round++) { + for (int i = 0; i < 100; i++) { + tracker.recordPendingAllocationForDatanode(dn, containers.get(i)); + } } + + // Should still only have 100 containers + assertEquals(100, tracker.getPendingContainerCount(dn)); } - private boolean addContainerToBucket(ContainerID containerID, DatanodeID dnID) { - return datanodeBuckets.get(dnID).add(containerID, dnID); + @Test + public void testMultiVolumeAccumulatedSpaceIsNotEnough() { + DatanodeDetails dn = datanodes.get(0); + long containerSize = MAX_CONTAINER_SIZE; + + List reports = new ArrayList<>(); + reports.add(createStorageReport(dn, 100 * containerSize, containerSize / 4, 0)); + reports.add(createStorageReport(dn, 100 * containerSize, containerSize / 4, 0)); + reports.add(createStorageReport(dn, 100 * containerSize, containerSize / 2, 0)); + DatanodeInfo dnInfo = new DatanodeInfo(dn, NodeStatus.inServiceHealthy(), null); + dnInfo.updateStorageReports(reports); + + assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); } - /** - * 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) { - Objects.requireNonNull(node, "node == null"); - Objects.requireNonNull(containerID, "containerID == null"); + @Test + public void testMultiVolumeWithPendingAllocation() { + DatanodeDetails dn = datanodes.get(0); + long containerSize = MAX_CONTAINER_SIZE; - DatanodeID dnID = node.getID(); - boolean removed = removeContainerFromBucket(containerID, dnID); + // Remaining space available for 3 containers across all the volumes + tracker.recordPendingAllocationForDatanode(dn, containers.get(0)); + tracker.recordPendingAllocationForDatanode(dn, containers.get(1)); - if (removed && metrics != null) { - metrics.incNumPendingContainersRemoved(); - } - } + List reports = new ArrayList<>(); + reports.add(createStorageReport(dn, 100 * containerSize, containerSize, 0)); + reports.add(createStorageReport(dn, 50 * containerSize, containerSize, 0)); + reports.add(createStorageReport(dn, 100 * containerSize, containerSize, 0)); + DatanodeInfo dnInfo = new DatanodeInfo(dn, NodeStatus.inServiceHealthy(), null); + dnInfo.updateStorageReports(reports); + // Remaining space available for 1 container across all the volume after 2 container allocation - private boolean removeContainerFromBucket(ContainerID containerID, DatanodeID dnID) { - return datanodeBuckets.get(dnID).remove(containerID, dnID); - } + assertTrue(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); - /** - * Number of pending container allocations for this datanode (union of current and previous - * windows). This call may advance the internal tumbling window if the roll interval has elapsed. - * - * @param node The DataNode - * @return Pending container count - */ - public long getPendingContainerCount(DatanodeDetails node) { - Objects.requireNonNull(node, "node == null"); - return datanodeBuckets.get(node).getCount(); + tracker.recordPendingAllocationForDatanode(dn, containers.get(2)); + // Remaining space available for 0 container across all the volume + assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); } - /** - * Whether container is in the current or previous window for this datanode. - */ - @VisibleForTesting - public boolean containsPendingContainer(DatanodeDetails node, ContainerID containerID) { - Objects.requireNonNull(node, "node == null"); - Objects.requireNonNull(containerID, "containerID == null"); - return datanodeBuckets.get(node).contains(containerID); + @Test + public void testMultiVolumeWithCommittedBytes() { + DatanodeDetails dn = datanodes.get(0); + long containerSize = MAX_CONTAINER_SIZE; + + List reports = new ArrayList<>(); + reports.add(createStorageReport(dn, 100 * containerSize, 6 * containerSize, 5 * containerSize)); + reports.add(createStorageReport(dn, 50 * containerSize, 3 * containerSize, 3 * containerSize)); + DatanodeInfo dnInfo = new DatanodeInfo(dn, NodeStatus.inServiceHealthy(), null); + dnInfo.updateStorageReports(reports); + // Remaining space available for 1 container across all the volume considering committed bytes + + assertTrue(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); + tracker.recordPendingAllocationForDatanode(dn, containers.get(0)); + // Remaining space available for 0 container across all the volume considering + // committed bytes and container allocation + assertFalse(tracker.hasEffectiveAllocatableSpaceForNewContainer(dn, dnInfo)); } - @VisibleForTesting - public SCMNodeMetrics getMetrics() { - return metrics; + private StorageReportProto createStorageReport(DatanodeDetails dn, long capacity, long remaining, long committed) { + return HddsTestUtils.createStorageReports(dn.getID(), capacity, remaining, committed).get(0); } } 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 4086c9484729..157ebda21042 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; @@ -354,9 +353,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 c874233b06949bff9f56f4230473215dbeac2a3f Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 20 Apr 2026 12:42:32 +0530 Subject: [PATCH 12/14] Rebase and update fix --- .../scm/container/ContainerManagerImpl.java | 4 +- .../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 -- .../hdds/scm/pipeline/PipelineManager.java | 7 ++-- .../scm/pipeline/PipelineManagerImpl.java | 4 +- .../hdds/scm/container/MockNodeManager.java | 7 ++-- .../scm/container/SimpleMockNodeManager.java | 5 ++- .../container/TestContainerManagerImpl.java | 7 ++-- .../hdds/scm/node/TestContainerPlacement.java | 3 +- .../scm/pipeline/MockPipelineManager.java | 4 +- .../scm/pipeline/TestPipelineManagerImpl.java | 13 +++---- ...estPendingContainerTrackerIntegration.java | 39 ++++++++----------- 15 files changed, 53 insertions(+), 69 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 a265a810c1a5..adbd00c4a53a 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 @@ -252,7 +252,7 @@ private ContainerInfo createContainer(Pipeline pipeline, String owner) private ContainerInfo allocateContainer(final Pipeline pipeline, final String owner) throws IOException { - if (!pipelineManager.hasEnoughSpace(pipeline, maxContainerSize)) { + if (!pipelineManager.hasEnoughSpace(pipeline)) { LOG.debug("Cannot allocate a new container because pipeline {} does not have the required space {}.", pipeline, maxContainerSize); return null; @@ -287,8 +287,6 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); // Record pending allocation - tracks containers scheduled but not yet written pendingContainerTracker.recordPendingAllocation(pipeline, containerID); - LOG.debug("Allocated container {} on pipeline {}. Recorded as pending on {} DataNodes", - containerID, pipeline.getId(), pipeline.getNodes().size()); 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 29d97100dd0d..66861e6756f4 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 @@ -432,5 +432,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 e118e6ab3553..f015e172e7ad 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 @@ -251,11 +251,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) { @@ -263,7 +260,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; @@ -1145,8 +1142,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/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 6a448d6c88df..6f50f550019a 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 @@ -213,13 +213,12 @@ void reinitialize(Table pipelineStore) void releaseWriteLock(); /** - * Checks whether all Datanodes in the specified pipeline have greater than the specified space, containerSize. + * Checks whether all Datanodes in the specified pipeline have greater than the container size. * @param pipeline pipeline to check - * @param containerSize the required amount of space - * @return false if all the volumes on any Datanode in the pipeline have space less than equal to the specified + * @return false if all the volumes on any Datanode in the pipeline have space less than equal to the * containerSize, otherwise true */ - boolean hasEnoughSpace(Pipeline pipeline, long containerSize); + boolean hasEnoughSpace(Pipeline pipeline); 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 b1de6e9569e7..6e78a83bad69 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 @@ -634,9 +634,9 @@ private boolean isOpenWithUnregisteredNodes(Pipeline pipeline) { } @Override - public boolean hasEnoughSpace(Pipeline pipeline, long containerSize) { + public boolean hasEnoughSpace(Pipeline pipeline) { for (DatanodeDetails node : pipeline.getNodes()) { - if (!nodeManager.hasSpaceForNewContainerAllocation(node, containerSize)) { + if (!nodeManager.hasSpaceForNewContainerAllocation(node)) { 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 8e94a384a0bd..dbf2ae4f7481 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 @@ -945,20 +945,21 @@ public void setNumHealthyVolumes(int value) { @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) { DatanodeInfo info = getDatanodeInfo(node); if (info == null) { return false; } return getPendingContainerTracker() - .hasEffectiveAllocatableSpaceForNewContainer(node, info, containerSize); + .hasEffectiveAllocatableSpaceForNewContainer(node, info); } /** 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 8ab456d3f2d7..97ce8cbe3c67 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 @@ -439,14 +439,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-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 7b12a0fc89ed..2e4801338e46 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 @@ -26,7 +26,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -104,7 +103,7 @@ 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), anyLong()); + doReturn(true).when(pipelineManager).hasEnoughSpace(any(Pipeline.class)); pipelineManager.createPipeline(RatisReplicationConfig.getInstance( ReplicationFactor.THREE)); @@ -143,7 +142,7 @@ void testAllocateContainer() throws Exception { */ @Test public void testGetMatchingContainerReturnsNullWhenNotEnoughSpaceInDatanodes() throws IOException { - doReturn(false).when(pipelineManager).hasEnoughSpace(any(), anyLong()); + doReturn(false).when(pipelineManager).hasEnoughSpace(any()); long sizeRequired = 256 * 1024 * 1024; // 256 MB Pipeline pipeline = pipelineManager.getPipelines().iterator().next(); @@ -168,7 +167,7 @@ public void testGetMatchingContainerReturnsContainerWhenEnoughSpaceInDatanodes() // create a spy to mock hasEnoughSpace to always return true PipelineManager spyPipelineManager = spy(pipelineManager); doReturn(true).when(spyPipelineManager) - .hasEnoughSpace(any(Pipeline.class), anyLong()); + .hasEnoughSpace(any(Pipeline.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 ea63a81c8fe4..3e8d41d35efd 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 @@ -26,7 +26,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -161,7 +160,7 @@ SCMNodeManager createNodeManager(OzoneConfiguration config) { ContainerManager createContainerManager() throws IOException { pipelineManager = spy(pipelineManager); - doReturn(true).when(pipelineManager).hasEnoughSpace(any(), anyLong()); + doReturn(true).when(pipelineManager).hasEnoughSpace(any()); return new ContainerManagerImpl(conf, scmhaManager, sequenceIdGen, pipelineManager, 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 157ebda21042..b92f09401cd7 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java @@ -334,9 +334,9 @@ public boolean isPipelineCreationFrozen() { } @Override - public boolean hasEnoughSpace(Pipeline pipeline, long containerSize) { + public boolean hasEnoughSpace(Pipeline pipeline) { for (DatanodeDetails node : pipeline.getNodes()) { - if (!nodeManager.hasSpaceForNewContainerAllocation(node, containerSize)) { + if (!nodeManager.hasSpaceForNewContainerAllocation(node)) { return false; } } 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 d7a0e682fab4..6b0b91eec822 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 @@ -938,7 +938,7 @@ public void testCreatePipelineForRead() throws IOException { } /** - * {@link PipelineManager#hasEnoughSpace(Pipeline, long)} should return false if all the + * {@link PipelineManager#hasEnoughSpace(Pipeline)} should return false if all the * volumes on any Datanode in the pipeline have less than equal to the space required for creating a new container. */ @Test @@ -979,11 +979,10 @@ public void testHasEnoughSpace() throws IOException { } doAnswer(invocation -> { DatanodeDetails dn = invocation.getArgument(0); - long cs = invocation.getArgument(1); DatanodeInfo info = mockedNodeManager.getDatanodeInfo(dn); - return SCMCommonPlacementPolicy.hasEnoughSpace(info, 0, cs); - }).when(mockedNodeManager).hasSpaceForNewContainerAllocation(any(DatanodeDetails.class), anyLong()); - assertTrue(pipelineManager.hasEnoughSpace(pipeline, containerSize)); + return SCMCommonPlacementPolicy.hasEnoughSpace(info, 0, containerSize); + }).when(mockedNodeManager).hasSpaceForNewContainerAllocation(any(DatanodeDetails.class)); + assertTrue(pipelineManager.hasEnoughSpace(pipeline)); // Case 2: One node does not have enough space. /* @@ -997,13 +996,13 @@ public void testHasEnoughSpace() throws IOException { DatanodeInfo datanodeInfo = datanodeInfoList.get(0); datanodeInfo.updateStorageReports(HddsTestUtils.createStorageReports(datanodeInfo.getID(), 200L, 120L, 20L)); - assertFalse(pipelineManager.hasEnoughSpace(pipeline, containerSize)); + assertFalse(pipelineManager.hasEnoughSpace(pipeline)); // Case 3: All nodes do not have enough space. for (DatanodeInfo info : datanodeInfoList) { info.updateStorageReports(HddsTestUtils.createStorageReports(info.getID(), 200L, 100L, 20L)); } - assertFalse(pipelineManager.hasEnoughSpace(pipeline, containerSize)); + assertFalse(pipelineManager.hasEnoughSpace(pipeline)); } private Set createContainerReplicasList( 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 973d2be32f677e46c733866e455f85625e942ad0 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 20 Apr 2026 14:34:26 +0530 Subject: [PATCH 13/14] Refactor code --- .../hdds/scm/container/ContainerManagerImpl.java | 12 ++---------- .../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/pipeline/PipelineManagerImpl.java | 7 +++++++ .../hdds/scm/server/StorageContainerManager.java | 2 +- .../hadoop/hdds/scm/block/TestBlockManager.java | 3 +-- .../scm/container/TestContainerManagerImpl.java | 5 ++--- .../hdds/scm/node/TestContainerPlacement.java | 3 +-- .../hdds/scm/pipeline/MockPipelineManager.java | 4 ++++ .../hdds/scm/safemode/TestSCMSafeModeManager.java | 3 +-- .../ozone/recon/scm/ReconContainerManager.java | 6 ++---- .../scm/ReconStorageContainerManagerFacade.java | 2 +- .../scm/AbstractReconContainerManagerTest.java | 3 +-- 15 files changed, 46 insertions(+), 28 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 adbd00c4a53a..3daf07afa641 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 @@ -26,7 +26,6 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; -import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.concurrent.locks.Lock; @@ -45,8 +44,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.node.PendingContainerTracker; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.utils.db.Table; @@ -83,7 +80,6 @@ public class ContainerManagerImpl implements ContainerManager { private final Random random = new Random(); private final long maxContainerSize; - private final PendingContainerTracker pendingContainerTracker; /** * @@ -94,8 +90,7 @@ public ContainerManagerImpl( final SequenceIdGenerator sequenceIdGen, final PipelineManager pipelineManager, final Table containerStore, - final ContainerReplicaPendingOps containerReplicaPendingOps, - final NodeManager nodeManager) + final ContainerReplicaPendingOps containerReplicaPendingOps) throws IOException { // Introduce builder for this class? this.lock = new ReentrantLock(); @@ -115,8 +110,6 @@ public ContainerManagerImpl( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES); this.scmContainerManagerMetrics = SCMContainerManagerMetrics.create(); - this.pendingContainerTracker = Objects.requireNonNull( - nodeManager.getPendingContainerTracker(), "pendingContainerTracker"); } @Override @@ -285,8 +278,7 @@ private ContainerInfo allocateContainer(final Pipeline pipeline, containerStateManager.addContainer(containerInfoBuilder.build()); scmContainerManagerMetrics.incNumSuccessfulCreateContainers(); - // Record pending allocation - tracks containers scheduled but not yet written - pendingContainerTracker.recordPendingAllocation(pipeline, containerID); + pipelineManager.recordPendingAllocation(pipeline, containerID); return containerStateManager.getContainer(containerID); } 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 66861e6756f4..039bb71ae877 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 @@ -433,4 +433,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 f015e172e7ad..7c014addb4f9 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 @@ -267,6 +267,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 6f50f550019a..ecfe9b1ef5f4 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 @@ -220,6 +220,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/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index 6e78a83bad69..2e3c522677a2 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 @@ -643,6 +643,13 @@ public boolean hasEnoughSpace(Pipeline pipeline) { return true; } + @Override + public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { + for (DatanodeDetails node : pipeline.getNodes()) { + nodeManager.recordPendingAllocationForDatanode(node, containerID); + } + } + /** * Schedules a fixed interval job to create 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 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/container/TestContainerManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java index 2e4801338e46..ca366d115484 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 @@ -111,8 +111,7 @@ void setUp() throws Exception { pendingOpsMock = mock(ContainerReplicaPendingOps.class); containerManager = new ContainerManagerImpl(conf, scmhaManager, sequenceIdGen, pipelineManager, - SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock, - nodeManager); + SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); } @AfterEach @@ -174,7 +173,7 @@ public void testGetMatchingContainerReturnsContainerWhenEnoughSpaceInDatanodes() OzoneConfiguration conf = SCMTestUtils.getConf(tempDir); ContainerManager manager = new ContainerManagerImpl(conf, scmhaManager, sequenceIdGen, spyPipelineManager, - SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock, nodeManager); + SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock); Pipeline pipeline = spyPipelineManager.getPipelines().iterator().next(); // the pipeline has no existing containers, so a new container gets allocated in getMatchingContainer 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 3e8d41d35efd..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 @@ -166,8 +166,7 @@ ContainerManager createContainerManager() scmhaManager, sequenceIdGen, pipelineManager, SCMDBDefinition.CONTAINERS.getTable(dbStore), new ContainerReplicaPendingOps( - Clock.system(ZoneId.systemDefault()), null), - nodeManager); + Clock.system(ZoneId.systemDefault()), null)); } /** 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 b92f09401cd7..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 @@ -343,6 +343,10 @@ public boolean hasEnoughSpace(Pipeline pipeline) { return true; } + @Override + public void recordPendingAllocation(Pipeline pipeline, ContainerID containerID) { + } + @Override public int openContainerLimit(List datanodes) { // For tests that do not care about this limit, return a large value. 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 656e77128cf10802dc952a3f928ea3e0044d3f77 Mon Sep 17 00:00:00 2001 From: ashishk Date: Mon, 20 Apr 2026 14:43:30 +0530 Subject: [PATCH 14/14] Move member to local variable --- .../hadoop/hdds/scm/container/TestContainerManagerImpl.java | 3 +-- .../apache/hadoop/hdds/scm/node/TestContainerPlacement.java | 3 +-- 2 files changed, 2 insertions(+), 4 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 ca366d115484..68dcc634a5e3 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 @@ -83,7 +83,6 @@ public class TestContainerManagerImpl { private SequenceIdGenerator sequenceIdGen; private ContainerReplicaPendingOps pendingOpsMock; private PipelineManager pipelineManager; - private NodeManager nodeManager; @BeforeAll static void init() { @@ -96,7 +95,7 @@ void setUp() throws Exception { final OzoneConfiguration conf = SCMTestUtils.getConf(testDir); dbStore = DBStoreBuilder.createDBStore(conf, SCMDBDefinition.get()); scmhaManager = SCMHAManagerStub.getInstance(true); - nodeManager = new MockNodeManager(true, 10); + NodeManager nodeManager = new MockNodeManager(true, 10); sequenceIdGen = new SequenceIdGenerator( conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore)); PipelineManager base = new MockPipelineManager(dbStore, scmhaManager, nodeManager); 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(