rakeshadr commented on code in PR #10000: URL: https://github.com/apache/ozone/pull/10000#discussion_r3044295093
########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java: ########## @@ -0,0 +1,425 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +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.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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; + +/** + * 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: + * <li>Each DataNode has two sets: <b>currentWindow</b> and <b>previousWindow</b></li> + * <li>New allocations go into <b>currentWindow</b></li> + * <li>Every <b>ROLL_INTERVAL</b> (default 5 minutes): + * <ul> + * <li>previousWindow = currentWindow (shift)</li> + * <li>currentWindow = new empty set (reset)</li> + * <li>Old previousWindow is discarded (automatic aging)</li> + * </ul> + * </li> + * <li>When checking pending: return <b>union</b> of currentWindow + previousWindow</li> + * + * + * Example Timeline: + * <pre> + * 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) + * </pre> + * + * <p>Thread-safety: all mutations and reads for a given datanode run under + * {@code synchronized (bucket)} on that datanode's {@link TwoWindowBucket}, including + * map insert/remove, so concurrent record/remove/report paths cannot interleave or orphan + * a bucket after it was dropped from the map.</p> + */ +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: 5 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap<UUID, TwoWindowBucket> datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set<ContainerID> currentWindow = new HashSet<>(); + private Set<ContainerID> previousWindow = new HashSet<>(); + 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) { Review Comment: Now, its making single-step roll, please do multi-step roll to stay clock-aligned regardless of when the roll fires. should be `lastRollTime += rollIntervalMs` ``` while (now - lastRollTime >= rollIntervalMs) { // Shift: current becomes previous previousWindow = currentWindow; // Reset: new empty current window currentWindow = new HashSet<>(); lastRollTime += rollIntervalMs; // clock-aligned, not drifting LOG.debug("Rolled window. Previous window size: {}, Current window reset to empty", previousWindow.size()); if (getAllPending().size() <= 0 ) { // exit if both windows got rollover lastRollTime = now; break; } } ``` ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java: ########## @@ -0,0 +1,425 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +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.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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; + +/** + * 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: + * <li>Each DataNode has two sets: <b>currentWindow</b> and <b>previousWindow</b></li> + * <li>New allocations go into <b>currentWindow</b></li> + * <li>Every <b>ROLL_INTERVAL</b> (default 5 minutes): + * <ul> + * <li>previousWindow = currentWindow (shift)</li> + * <li>currentWindow = new empty set (reset)</li> + * <li>Old previousWindow is discarded (automatic aging)</li> + * </ul> + * </li> + * <li>When checking pending: return <b>union</b> of currentWindow + previousWindow</li> + * + * + * Example Timeline: + * <pre> + * 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) + * </pre> + * + * <p>Thread-safety: all mutations and reads for a given datanode run under + * {@code synchronized (bucket)} on that datanode's {@link TwoWindowBucket}, including + * map insert/remove, so concurrent record/remove/report paths cannot interleave or orphan + * a bucket after it was dropped from the map.</p> + */ +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: 5 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap<UUID, TwoWindowBucket> datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set<ContainerID> currentWindow = new HashSet<>(); + private Set<ContainerID> previousWindow = new HashSet<>(); + 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 = new HashSet<>(); + 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<ContainerID> getAllPending() { + Set<ContainerID> 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(); + } + + /** + * Count of pending containers in both windows. + */ + synchronized int getCount() { + return currentWindow.size() + previousWindow.size(); + } + } + + public PendingContainerTracker(long maxContainerSize) { + this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes + } + + public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, + SCMNodeMetrics 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); + } + + /** + * Whether the datanode can fit another container of {@code containerSize} after accounting for + * SCM pending allocations for {@code node} (this tracker) and usable space on {@code datanodeInfo}. + * Combines {@link #getPendingAllocationSize} with the per-disk slot check in one call. + * + * @param node identity used to look up pending allocations (same DN as {@code datanodeInfo}) + * @param datanodeInfo storage reports for the datanode + * @param containerSize required container size in bytes (typically SCM max container size) + */ + /** + * Advances the two-window tumbling bucket for this datanode when the roll interval has elapsed. + * Call on periodic paths (e.g. node report) so windows age even when there are no new + * allocations or container reports touching this tracker. + */ + public void rollWindowsIfNeeded(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + bucket.rollIfNeeded(); + if (bucket.isEmpty()) { + datanodeBuckets.remove(uuid, bucket); + } + } + } + + public boolean hasEffectiveAllocatableSpaceForNewContainer( + DatanodeDetails node, DatanodeInfo datanodeInfo, long containerSize) { + if (node == null || datanodeInfo == null || containerSize <= 0) { + return false; + } + long pendingBytes = getPendingAllocationSize(node); + return hasAllocatableSpaceAfterPending(datanodeInfo, containerSize, pendingBytes); + } + + private static boolean hasAllocatableSpaceAfterPending( + DatanodeInfo datanodeInfo, long containerSize, long pendingAllocationBytes) { + List<StorageReportProto> storageReports = datanodeInfo.getStorageReports(); + if (storageReports == null || storageReports.isEmpty()) { + return false; + } + long effectiveAllocatableSpace = 0L; + for (StorageReportProto report : storageReports) { + long usableSpace = VolumeUsage.getUsableSpace(report); + long containersOnThisDisk = usableSpace / containerSize; + effectiveAllocatableSpace += containersOnThisDisk * containerSize; + if (effectiveAllocatableSpace - pendingAllocationBytes >= containerSize) { + return true; + } + } + return false; + } + + /** + * Drops all pending allocation state for a datanode (e.g. stale/dead cleanup). + */ + public void clearPendingForDatanode(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + datanodeBuckets.remove(uuid, bucket); Review Comment: ConcurrentHashMap.remove(key, value) is already atomic. The synchronized(bucket) doesn't protect any bucket-internal state here, right? You can remove this unncessary lock? ########## hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestPendingContainerTrackerIntegration.java: ########## @@ -0,0 +1,263 @@ +/* + * 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.node.PendingContainerTracker; +import org.apache.hadoop.hdds.scm.node.SCMNodeManager; +import org.apache.hadoop.hdds.scm.node.SCMNodeMetrics; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.server.StorageContainerManager; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.TestDataUtil; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Integration tests for PendingContainerTracker. + */ +@Timeout(300) +public class TestPendingContainerTrackerIntegration { + + private static final Logger LOG = + LoggerFactory.getLogger(TestPendingContainerTrackerIntegration.class); + private MiniOzoneCluster cluster; + private StorageContainerManager scm; + private OzoneClient client; + private ContainerManager containerManager; + private PendingContainerTracker pendingTracker; + private SCMNodeMetrics metrics; + private OzoneBucket bucket; + + @BeforeEach + public void setup() throws Exception { + OzoneConfiguration conf = new OzoneConfiguration(); + + conf.set(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, "60s"); + + // Reduce heartbeat interval for faster container reports + conf.set(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, "10s"); + + conf.set("ozone.scm.container.size", "100MB"); + conf.set("ozone.scm.pipeline.owner.container.count", "1"); + conf.set("ozone.scm.pipeline.per.metadata.disk", "1"); + conf.set("ozone.scm.datanode.pipeline.limit", "1"); + + cluster = MiniOzoneCluster.newBuilder(conf) + .setNumDatanodes(3) + .build(); + cluster.waitForClusterToBeReady(); + cluster.waitTobeOutOfSafeMode(); + + scm = cluster.getStorageContainerManager(); + containerManager = scm.getContainerManager(); + client = cluster.newClient(); + + // Create bucket for testing + bucket = TestDataUtil.createVolumeAndBucket(client); + + SCMNodeManager nodeManager = (SCMNodeManager) scm.getScmNodeManager(); + assertNotNull(nodeManager); + pendingTracker = nodeManager.getPendingContainerTracker(); + assertNotNull(pendingTracker, "PendingContainerTracker should be initialized"); + metrics = pendingTracker.getMetrics(); + + LOG.info("Test setup complete - ICR interval: 5s, Heartbeat interval: 1s"); + } + + @AfterEach + public void cleanup() throws Exception { + if (client != null) { + client.close(); + } + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Test: Write key → Container allocation → Pending tracked → ICR → Pending removed. + */ + @Test + public void testKeyWriteRecordsPendingAndICRRemovesIt() throws Exception { + long initialAdded = metrics.getNumPendingContainersAdded(); + long initialRemoved = metrics.getNumPendingContainersRemoved(); + + // Allocate a container directly + 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<DatanodeDetails> 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<ContainerID> pendingContainers = pendingTracker.getPendingContainers(dn); + assertThat(pendingContainers.contains(container.containerID())); Review Comment: Should this be `assertThat(pendingContainers).contains(container.containerID());` ? Please visit all the occurrences of assertThat() in this file and fix it. ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java: ########## @@ -0,0 +1,425 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +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.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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; + +/** + * 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: + * <li>Each DataNode has two sets: <b>currentWindow</b> and <b>previousWindow</b></li> + * <li>New allocations go into <b>currentWindow</b></li> + * <li>Every <b>ROLL_INTERVAL</b> (default 5 minutes): + * <ul> + * <li>previousWindow = currentWindow (shift)</li> + * <li>currentWindow = new empty set (reset)</li> + * <li>Old previousWindow is discarded (automatic aging)</li> + * </ul> + * </li> + * <li>When checking pending: return <b>union</b> of currentWindow + previousWindow</li> + * + * + * Example Timeline: + * <pre> + * 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) + * </pre> + * + * <p>Thread-safety: all mutations and reads for a given datanode run under + * {@code synchronized (bucket)} on that datanode's {@link TwoWindowBucket}, including + * map insert/remove, so concurrent record/remove/report paths cannot interleave or orphan + * a bucket after it was dropped from the map.</p> + */ +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: 5 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap<UUID, TwoWindowBucket> datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set<ContainerID> currentWindow = new HashSet<>(); + private Set<ContainerID> previousWindow = new HashSet<>(); + 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 = new HashSet<>(); + 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<ContainerID> getAllPending() { + Set<ContainerID> 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(); + } + + /** + * Count of pending containers in both windows. + */ + synchronized int getCount() { + return currentWindow.size() + previousWindow.size(); + } + } + + public PendingContainerTracker(long maxContainerSize) { + this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes + } + + public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, + SCMNodeMetrics 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); + } + + /** + * Whether the datanode can fit another container of {@code containerSize} after accounting for + * SCM pending allocations for {@code node} (this tracker) and usable space on {@code datanodeInfo}. + * Combines {@link #getPendingAllocationSize} with the per-disk slot check in one call. + * + * @param node identity used to look up pending allocations (same DN as {@code datanodeInfo}) + * @param datanodeInfo storage reports for the datanode + * @param containerSize required container size in bytes (typically SCM max container size) + */ + /** + * Advances the two-window tumbling bucket for this datanode when the roll interval has elapsed. + * Call on periodic paths (e.g. node report) so windows age even when there are no new + * allocations or container reports touching this tracker. + */ + public void rollWindowsIfNeeded(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + bucket.rollIfNeeded(); + if (bucket.isEmpty()) { + datanodeBuckets.remove(uuid, bucket); + } + } + } + + public boolean hasEffectiveAllocatableSpaceForNewContainer( + DatanodeDetails node, DatanodeInfo datanodeInfo, long containerSize) { + if (node == null || datanodeInfo == null || containerSize <= 0) { + return false; + } + long pendingBytes = getPendingAllocationSize(node); + return hasAllocatableSpaceAfterPending(datanodeInfo, containerSize, pendingBytes); + } + + private static boolean hasAllocatableSpaceAfterPending( + DatanodeInfo datanodeInfo, long containerSize, long pendingAllocationBytes) { + List<StorageReportProto> storageReports = datanodeInfo.getStorageReports(); + if (storageReports == null || storageReports.isEmpty()) { + return false; + } + long effectiveAllocatableSpace = 0L; + for (StorageReportProto report : storageReports) { + long usableSpace = VolumeUsage.getUsableSpace(report); + long containersOnThisDisk = usableSpace / containerSize; + effectiveAllocatableSpace += containersOnThisDisk * containerSize; + if (effectiveAllocatableSpace - pendingAllocationBytes >= containerSize) { + return true; + } + } + return false; + } + + /** + * Drops all pending allocation state for a datanode (e.g. stale/dead cleanup). + */ + public void clearPendingForDatanode(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + datanodeBuckets.remove(uuid, bucket); + } + LOG.debug("Cleared pending container allocations for datanode {}", + node.getUuidString()); + } + + /** + * 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; + } + + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.computeIfAbsent( + uuid, + k -> new TwoWindowBucket(rollIntervalMs) + ); + + synchronized (bucket) { + bucket.rollIfNeeded(); + boolean added = bucket.add(containerID); + if (!bucket.isEmpty()) { + datanodeBuckets.put(uuid, bucket); + } + LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", + containerID, node.getUuidString(), added, bucket.getCount()); + + 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; + } + + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + + synchronized (bucket) { + bucket.rollIfNeeded(); + + boolean removed = bucket.remove(containerID); + LOG.debug("Removed pending container {} from DataNode {}. Removed={}, Remaining={}", + containerID, node.getUuidString(), removed, bucket.getCount()); + + if (removed && metrics != null) { + metrics.incNumPendingContainersRemoved(); + } + + if (bucket.isEmpty()) { + LOG.debug("Cleanup pending bucket containerID {}", containerID); + datanodeBuckets.remove(uuid, bucket); + } + } + } + + /** + * Bytes of SCM-side pending container allocations for this datanode (count × configured max + * container size). For whether a new container can be placed, prefer + * {@link #hasEffectiveAllocatableSpaceForNewContainer}. + * + * @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()); + if (bucket == null) { + return 0; + } + + synchronized (bucket) { + bucket.rollIfNeeded(); Review Comment: Can you update the javadoc saying that, ``` /** * Bytes of SCM-side pending container allocations for this datanode (count × configured max * container size). For whether a new container can be placed, prefer * {@link #hasEffectiveAllocatableSpaceForNewContainer}. * * <p>Note: this call may advance the internal tumbling window if the roll interval has elapsed, * ensuring the returned value reflects the most up-to-date pending state.</p> * * @param node The DataNode * @return Total bytes of pending container allocations */ ``` ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java: ########## @@ -0,0 +1,425 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +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.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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; + +/** + * 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: + * <li>Each DataNode has two sets: <b>currentWindow</b> and <b>previousWindow</b></li> + * <li>New allocations go into <b>currentWindow</b></li> + * <li>Every <b>ROLL_INTERVAL</b> (default 5 minutes): + * <ul> + * <li>previousWindow = currentWindow (shift)</li> + * <li>currentWindow = new empty set (reset)</li> + * <li>Old previousWindow is discarded (automatic aging)</li> + * </ul> + * </li> + * <li>When checking pending: return <b>union</b> of currentWindow + previousWindow</li> + * + * + * Example Timeline: + * <pre> + * 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) + * </pre> + * + * <p>Thread-safety: all mutations and reads for a given datanode run under + * {@code synchronized (bucket)} on that datanode's {@link TwoWindowBucket}, including + * map insert/remove, so concurrent record/remove/report paths cannot interleave or orphan + * a bucket after it was dropped from the map.</p> + */ +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: 5 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap<UUID, TwoWindowBucket> datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set<ContainerID> currentWindow = new HashSet<>(); + private Set<ContainerID> previousWindow = new HashSet<>(); + 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 = new HashSet<>(); + 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<ContainerID> getAllPending() { + Set<ContainerID> 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(); + } + + /** + * Count of pending containers in both windows. + */ + synchronized int getCount() { + return currentWindow.size() + previousWindow.size(); + } + } + + public PendingContainerTracker(long maxContainerSize) { + this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes + } + + public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, + SCMNodeMetrics 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); + } + + /** + * Whether the datanode can fit another container of {@code containerSize} after accounting for + * SCM pending allocations for {@code node} (this tracker) and usable space on {@code datanodeInfo}. + * Combines {@link #getPendingAllocationSize} with the per-disk slot check in one call. + * + * @param node identity used to look up pending allocations (same DN as {@code datanodeInfo}) + * @param datanodeInfo storage reports for the datanode + * @param containerSize required container size in bytes (typically SCM max container size) + */ + /** + * Advances the two-window tumbling bucket for this datanode when the roll interval has elapsed. + * Call on periodic paths (e.g. node report) so windows age even when there are no new + * allocations or container reports touching this tracker. + */ + public void rollWindowsIfNeeded(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + bucket.rollIfNeeded(); + if (bucket.isEmpty()) { + datanodeBuckets.remove(uuid, bucket); + } + } + } + + public boolean hasEffectiveAllocatableSpaceForNewContainer( + DatanodeDetails node, DatanodeInfo datanodeInfo, long containerSize) { + if (node == null || datanodeInfo == null || containerSize <= 0) { + return false; + } + long pendingBytes = getPendingAllocationSize(node); + return hasAllocatableSpaceAfterPending(datanodeInfo, containerSize, pendingBytes); + } + + private static boolean hasAllocatableSpaceAfterPending( + DatanodeInfo datanodeInfo, long containerSize, long pendingAllocationBytes) { + List<StorageReportProto> storageReports = datanodeInfo.getStorageReports(); + if (storageReports == null || storageReports.isEmpty()) { + return false; + } + long effectiveAllocatableSpace = 0L; + for (StorageReportProto report : storageReports) { + long usableSpace = VolumeUsage.getUsableSpace(report); + long containersOnThisDisk = usableSpace / containerSize; + effectiveAllocatableSpace += containersOnThisDisk * containerSize; + if (effectiveAllocatableSpace - pendingAllocationBytes >= containerSize) { + return true; + } + } + return false; + } + + /** + * Drops all pending allocation state for a datanode (e.g. stale/dead cleanup). + */ + public void clearPendingForDatanode(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + datanodeBuckets.remove(uuid, bucket); + } + LOG.debug("Cleared pending container allocations for datanode {}", + node.getUuidString()); + } + + /** + * 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; + } + + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.computeIfAbsent( Review Comment: Since `datanodeBuckets` is concurrent hashmap, can we simplify the logic instead of synchronized bucket and avoids unnecessary null check? ``` boolean added = addContainerToBucket(node.getUuid(), containerID); if (added && metrics != null) { metrics.incNumPendingContainersAdded(); } ``` ``` private boolean addContainerToBucket(UUID uuid, ContainerID containerID) { AtomicBoolean added = new AtomicBoolean(false); datanodeBuckets.compute(uuid, (k, existing) -> { TwoWindowBucket bucket = (existing != null) ? existing : new TwoWindowBucket(rollIntervalMs); bucket.rollIfNeeded(); added.set(bucket.add(containerID)); LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", containerID, uuid, added.get(), bucket.getCount()); return bucket; }); return added.get(); } ``` ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java: ########## @@ -0,0 +1,425 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +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.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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; + +/** + * 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: + * <li>Each DataNode has two sets: <b>currentWindow</b> and <b>previousWindow</b></li> + * <li>New allocations go into <b>currentWindow</b></li> + * <li>Every <b>ROLL_INTERVAL</b> (default 5 minutes): + * <ul> + * <li>previousWindow = currentWindow (shift)</li> + * <li>currentWindow = new empty set (reset)</li> + * <li>Old previousWindow is discarded (automatic aging)</li> + * </ul> + * </li> + * <li>When checking pending: return <b>union</b> of currentWindow + previousWindow</li> + * + * + * Example Timeline: + * <pre> + * 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) + * </pre> + * + * <p>Thread-safety: all mutations and reads for a given datanode run under + * {@code synchronized (bucket)} on that datanode's {@link TwoWindowBucket}, including + * map insert/remove, so concurrent record/remove/report paths cannot interleave or orphan + * a bucket after it was dropped from the map.</p> + */ +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: 5 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap<UUID, TwoWindowBucket> datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set<ContainerID> currentWindow = new HashSet<>(); + private Set<ContainerID> previousWindow = new HashSet<>(); + 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 = new HashSet<>(); + 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<ContainerID> getAllPending() { + Set<ContainerID> 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(); + } + + /** + * Count of pending containers in both windows. + */ + synchronized int getCount() { + return currentWindow.size() + previousWindow.size(); + } + } + + public PendingContainerTracker(long maxContainerSize) { + this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes + } + + public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, + SCMNodeMetrics 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); + } + + /** + * Whether the datanode can fit another container of {@code containerSize} after accounting for + * SCM pending allocations for {@code node} (this tracker) and usable space on {@code datanodeInfo}. + * Combines {@link #getPendingAllocationSize} with the per-disk slot check in one call. + * + * @param node identity used to look up pending allocations (same DN as {@code datanodeInfo}) + * @param datanodeInfo storage reports for the datanode + * @param containerSize required container size in bytes (typically SCM max container size) + */ + /** + * Advances the two-window tumbling bucket for this datanode when the roll interval has elapsed. + * Call on periodic paths (e.g. node report) so windows age even when there are no new + * allocations or container reports touching this tracker. + */ + public void rollWindowsIfNeeded(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + bucket.rollIfNeeded(); + if (bucket.isEmpty()) { + datanodeBuckets.remove(uuid, bucket); + } + } + } + + public boolean hasEffectiveAllocatableSpaceForNewContainer( + DatanodeDetails node, DatanodeInfo datanodeInfo, long containerSize) { + if (node == null || datanodeInfo == null || containerSize <= 0) { + return false; + } + long pendingBytes = getPendingAllocationSize(node); + return hasAllocatableSpaceAfterPending(datanodeInfo, containerSize, pendingBytes); + } + + private static boolean hasAllocatableSpaceAfterPending( + DatanodeInfo datanodeInfo, long containerSize, long pendingAllocationBytes) { + List<StorageReportProto> storageReports = datanodeInfo.getStorageReports(); + if (storageReports == null || storageReports.isEmpty()) { + return false; + } + long effectiveAllocatableSpace = 0L; + for (StorageReportProto report : storageReports) { + long usableSpace = VolumeUsage.getUsableSpace(report); + long containersOnThisDisk = usableSpace / containerSize; + effectiveAllocatableSpace += containersOnThisDisk * containerSize; + if (effectiveAllocatableSpace - pendingAllocationBytes >= containerSize) { + return true; + } + } + return false; + } + + /** + * Drops all pending allocation state for a datanode (e.g. stale/dead cleanup). + */ + public void clearPendingForDatanode(DatanodeDetails node) { + if (node == null) { + return; + } + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + synchronized (bucket) { + datanodeBuckets.remove(uuid, bucket); + } + LOG.debug("Cleared pending container allocations for datanode {}", + node.getUuidString()); + } + + /** + * 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; + } + + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.computeIfAbsent( + uuid, + k -> new TwoWindowBucket(rollIntervalMs) + ); + + synchronized (bucket) { + bucket.rollIfNeeded(); + boolean added = bucket.add(containerID); + if (!bucket.isEmpty()) { + datanodeBuckets.put(uuid, bucket); + } + LOG.debug("Recorded pending container {} on DataNode {}. Added={}, Total pending={}", + containerID, node.getUuidString(), added, bucket.getCount()); + + 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; + } + + UUID uuid = node.getUuid(); + TwoWindowBucket bucket = datanodeBuckets.get(uuid); + if (bucket == null) { + return; + } + + synchronized (bucket) { + bucket.rollIfNeeded(); + + boolean removed = bucket.remove(containerID); + LOG.debug("Removed pending container {} from DataNode {}. Removed={}, Remaining={}", + containerID, node.getUuidString(), removed, bucket.getCount()); + + if (removed && metrics != null) { + metrics.incNumPendingContainersRemoved(); + } + + if (bucket.isEmpty()) { + LOG.debug("Cleanup pending bucket containerID {}", containerID); + datanodeBuckets.remove(uuid, bucket); + } + } + } + + /** + * Bytes of SCM-side pending container allocations for this datanode (count × configured max + * container size). For whether a new container can be placed, prefer + * {@link #hasEffectiveAllocatableSpaceForNewContainer}. + * + * @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()); + if (bucket == null) { + return 0; + } + + synchronized (bucket) { + bucket.rollIfNeeded(); + 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<ContainerID> getPendingContainers(DatanodeDetails node) { + if (node == null) { + return Collections.emptySet(); + } + + TwoWindowBucket bucket = datanodeBuckets.get(node.getUuid()); + if (bucket == null) { + return Collections.emptySet(); + } + + synchronized (bucket) { + bucket.rollIfNeeded(); Review Comment: Can you update the javadoc saying that, ``` /** * Get the set of pending container IDs for a DataNode. * Returns union of current and previous windows. * Useful for debugging and monitoring. * * <p>Note: this call may advance the internal tumbling window if the roll interval has elapsed, * ensuring the returned set reflects the most up-to-date pending state.</p> * * @param node The DataNode * @return Set of pending container IDs */ ``` ########## hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestPendingContainerTracker.java: ########## @@ -0,0 +1,406 @@ +/* + * 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.node; + +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.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; + +/** + * 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)); + } + + /** + * 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. + */ + @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.getPendingContainers(dn1).size()); + assertThat(shortRollTracker.getPendingContainers(dn1)).contains(container1); + + // First roll: C1 moves from currentWindow to previousWindow; union still includes C1 + Thread.sleep(rollMs + 80); + shortRollTracker.rollWindowsIfNeeded(dn1); + assertEquals(1, shortRollTracker.getPendingContainers(dn1).size()); + assertThat(shortRollTracker.getPendingContainers(dn1)).contains(container1); + + // Second roll: prior previousWindow (holding C1) is dropped; C1 is no longer pending + Thread.sleep(rollMs + 80); + shortRollTracker.rollWindowsIfNeeded(dn1); + assertEquals(0, shortRollTracker.getPendingContainers(dn1).size()); + assertEquals(0L, shortRollTracker.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<ContainerID> pending = tracker.getPendingContainers(dn1); + + assertEquals(2, pending.size()); + assertThat(pending.contains(container1)); Review Comment: Should this be `assertThat(pending).contains(container1);` ? Please visit all the occurrences of assertThat() in this file and fix it. ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java: ########## @@ -0,0 +1,425 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +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.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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; + +/** + * 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: + * <li>Each DataNode has two sets: <b>currentWindow</b> and <b>previousWindow</b></li> + * <li>New allocations go into <b>currentWindow</b></li> + * <li>Every <b>ROLL_INTERVAL</b> (default 5 minutes): + * <ul> + * <li>previousWindow = currentWindow (shift)</li> + * <li>currentWindow = new empty set (reset)</li> + * <li>Old previousWindow is discarded (automatic aging)</li> + * </ul> + * </li> + * <li>When checking pending: return <b>union</b> of currentWindow + previousWindow</li> + * + * + * Example Timeline: + * <pre> + * 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) + * </pre> + * + * <p>Thread-safety: all mutations and reads for a given datanode run under + * {@code synchronized (bucket)} on that datanode's {@link TwoWindowBucket}, including + * map insert/remove, so concurrent record/remove/report paths cannot interleave or orphan + * a bucket after it was dropped from the map.</p> + */ +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: 5 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap<UUID, TwoWindowBucket> datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set<ContainerID> currentWindow = new HashSet<>(); + private Set<ContainerID> previousWindow = new HashSet<>(); + 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 = new HashSet<>(); + 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<ContainerID> getAllPending() { + Set<ContainerID> 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(); + } + + /** + * Count of pending containers in both windows. + */ + synchronized int getCount() { + return currentWindow.size() + previousWindow.size(); Review Comment: For safe coding, can you please do union, ``` synchronized int getCount() { return getAllPending().size() } ``` ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/PendingContainerTracker.java: ########## @@ -0,0 +1,425 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +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.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; +import org.apache.hadoop.hdds.scm.container.ContainerID; +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; + +/** + * 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: + * <li>Each DataNode has two sets: <b>currentWindow</b> and <b>previousWindow</b></li> + * <li>New allocations go into <b>currentWindow</b></li> + * <li>Every <b>ROLL_INTERVAL</b> (default 5 minutes): + * <ul> + * <li>previousWindow = currentWindow (shift)</li> + * <li>currentWindow = new empty set (reset)</li> + * <li>Old previousWindow is discarded (automatic aging)</li> + * </ul> + * </li> + * <li>When checking pending: return <b>union</b> of currentWindow + previousWindow</li> + * + * + * Example Timeline: + * <pre> + * 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) + * </pre> + * + * <p>Thread-safety: all mutations and reads for a given datanode run under + * {@code synchronized (bucket)} on that datanode's {@link TwoWindowBucket}, including + * map insert/remove, so concurrent record/remove/report paths cannot interleave or orphan + * a bucket after it was dropped from the map.</p> + */ +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: 5 minutes. + * Containers automatically age out after 2 × rollIntervalMs. + */ + private final long rollIntervalMs; + + /** + * Map of DataNode UUID to TwoWindowBucket. + */ + private final ConcurrentHashMap<UUID, TwoWindowBucket> datanodeBuckets; + + /** + * Maximum container size in bytes. + */ + private final long maxContainerSize; + + /** + * Metrics for tracking pending containers (same instance as {@link SCMNodeManager}'s node metrics). + */ + private final SCMNodeMetrics metrics; + + /** + * Two-window bucket for a single DataNode. + * Contains current and previous window sets, plus last roll timestamp. + */ + private static class TwoWindowBucket { + private Set<ContainerID> currentWindow = new HashSet<>(); + private Set<ContainerID> previousWindow = new HashSet<>(); + 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 = new HashSet<>(); + 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<ContainerID> getAllPending() { + Set<ContainerID> 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(); + } + + /** + * Count of pending containers in both windows. + */ + synchronized int getCount() { + return currentWindow.size() + previousWindow.size(); + } + } + + public PendingContainerTracker(long maxContainerSize) { + this(maxContainerSize, 10 * 60 * 1000, null); // Default 10 minutes + } + + public PendingContainerTracker(long maxContainerSize, long rollIntervalMs, + SCMNodeMetrics 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); + } + + /** Review Comment: This has two javadocs. I think the below one is meant for `hasEffectiveAllocatableSpaceForNewContainer` mthod, right?. Please move this javadoc there. ``` /** * Whether the datanode can fit another container of {@code containerSize} after accounting for * SCM pending allocations for {@code node} (this tracker) and usable space on {@code datanodeInfo}. * Combines {@link #getPendingAllocationSize} with the per-disk slot check in one call. * * @param node identity used to look up pending allocations (same DN as {@code datanodeInfo}) * @param datanodeInfo storage reports for the datanode * @param containerSize required container size in bytes (typically SCM max container size) */ ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
