rakeshadr commented on code in PR #10000:
URL: https://github.com/apache/ozone/pull/10000#discussion_r3021573986


##########
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:
+ *   <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 10 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: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)
+ * </pre>
+ *
+ */
+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<UUID, TwoWindowBucket> 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<ContainerID> currentWindow = ConcurrentHashMap.newKeySet();
+    private Set<ContainerID> 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<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();
+    }
+    
+    /**
+     * 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()) {

Review Comment:
   Potentially hits concurrency issue. Say two threads entered this block.
   
   `Thread-1 (removePendingAllocation)`: bucket.isEmpty(), returns true
   
   `Thread-2 (recordPendingAllocationForDatanode)`: computeIfAbsent(uuid) 
returns same bucket 
   reference (key still exists), calls bucket.add(containerID) and now the 
bucket will be non-empty
   
   `Thread-1: datanodeBuckets.remove(uuid, bucket)`, then removes the non-empty 
bucket and now the containerID will be in a detached bucket object, right?
   
   I think, we need to add synchronization to avoid detached bucket object.



##########
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<ContainerID> 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<ContainerID> 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() {

Review Comment:
   Do we have test scenario covering roll over?
   
   The two-window rolling behavior (container in previousWindow roll after 2× 
interval). Say, add C1 in currentWindow, then moves C1 to previousWindow, then 
wait for the roll over.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/IncrementalContainerReportHandler.java:
##########
@@ -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);

Review Comment:
   Say, DN is healthy, all containers confirmed, no new allocations → that DN's 
bucket never rolls even though heartbeats come every 30 seconds, right?
   
   ```
   t=0    Container C1 allocated → pending recorded in tracker
   
   t=60-120  FCR arrives from DN
             → cid = C1
             → alreadyInDn = expectedContainersInDatanode.remove(C1) = FALSE
             → !alreadyInDn = TRUE → removePendingAllocation called → 
rollIfNeeded fires ✓
             → C1 added to NM DN-set
   ```
   How abt rolls on every `processHeartbeat`, every 30 seconds regardless of 
container state changes ?



-- 
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]

Reply via email to