ArafatKhan2198 commented on code in PR #9243:
URL: https://github.com/apache/ozone/pull/9243#discussion_r2526056258


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java:
##########
@@ -99,21 +111,49 @@ public static boolean reprocess(OMMetadataManager 
omMetadataManager,
       // Get the appropriate table based on BucketLayout
       Table<String, OmKeyInfo> omKeyInfoTable = 
omMetadataManager.getKeyTable(bucketLayout);
 
-      // Iterate through the table and process keys
-      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>> 
keyIter = omKeyInfoTable.iterator()) {
-        while (keyIter.hasNext()) {
-          Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
-          handleKeyReprocess(kv.getKey(), kv.getValue(), containerKeyMap, 
containerKeyCountMap,
-              reconContainerMetadataManager);
-          omKeyCount++;
-
-          // Check and flush data if it reaches the batch threshold
-          if (!checkAndCallFlushToDB(containerKeyMap, 
containerKeyFlushToDBMaxThreshold,
-              reconContainerMetadataManager)) {
-            LOG.error("Failed to flush container key data for {}", taskName);
-            return false;
+      // Use fair lock to prevent write lock starvation when flushing
+      ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+      // Flag to coordinate flush attempts - prevents all threads from queuing 
for write lock
+      AtomicBoolean isFlushingInProgress = new AtomicBoolean(false);
+      
+      // Use parallel table iteration
+      Function<Table.KeyValue<String, OmKeyInfo>, Void> kvOperation = kv -> {
+        try {
+          try {
+            lock.readLock().lock();
+            handleKeyReprocess(kv.getKey(), kv.getValue(), containerKeyMap, 
containerKeyCountMap,
+                reconContainerMetadataManager);
+          } finally {
+            lock.readLock().unlock();
+          }
+          omKeyCount.incrementAndGet();
+          
+          // Only one thread should attempt flush to avoid blocking all workers
+          if (containerKeyMap.size() >= containerKeyFlushToDBMaxThreshold &&
+              isFlushingInProgress.compareAndSet(false, true)) {
+            try {
+              lock.writeLock().lock();
+              try {
+                if (!checkAndCallFlushToDB(containerKeyMap, 
containerKeyFlushToDBMaxThreshold,

Review Comment:
   Thanks, I have made the change.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.ozone.recon.tasks.util;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hdds.utils.db.Codec;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.rocksdb.LiveFileMetaData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to iterate through a table in parallel by breaking table into 
multiple iterators.
+ */
+public class ParallelTableIteratorOperation<K extends Comparable<K>, V> 
implements Closeable {
+  private final Table<K, V> table;
+  private final Codec<K> keyCodec;
+  private final ExecutorService iteratorExecutor;
+  private final ExecutorService valueExecutors;
+  private final int maxNumberOfVals;
+  private final OMMetadataManager metadataManager;
+  private final int maxIteratorTasks;
+  private final int maxWorkerTasks;
+  private final long logCountThreshold;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ParallelTableIteratorOperation.class);
+  public ParallelTableIteratorOperation(OMMetadataManager metadataManager, 
Table<K, V> table, Codec<K> keyCodec,
+                                        int iteratorCount, int workerCount, 
int maxNumberOfValsInMemory,
+                                        long logThreshold) {
+    this.table = table;
+    this.keyCodec = keyCodec;
+    this.metadataManager = metadataManager;
+    this.maxIteratorTasks = 2 * iteratorCount;
+    this.maxWorkerTasks = workerCount * 2;
+    this.iteratorExecutor = new ThreadPoolExecutor(iteratorCount, 
iteratorCount, 1, TimeUnit.MINUTES,
+                    new ArrayBlockingQueue<>(iteratorCount * 2),
+                    new ThreadPoolExecutor.CallerRunsPolicy());
+    this.valueExecutors = new ThreadPoolExecutor(workerCount, workerCount, 1, 
TimeUnit.MINUTES,
+            new ArrayBlockingQueue<>(workerCount * 2),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.maxNumberOfVals = Math.max(10, maxNumberOfValsInMemory / 
(workerCount));
+    this.logCountThreshold = logThreshold;
+  }
+
+
+  private List<K> getBounds(K startKey, K endKey) throws IOException {
+    Set<K> keys = new HashSet<>();
+
+    // Try to get SST file boundaries for optimal segmentation
+    // In test/mock environments, this may not be available
+    try {
+      RDBStore store = (RDBStore) this.metadataManager.getStore();
+      if (store != null && store.getDb() != null) {
+        List<LiveFileMetaData> sstFiles = store.getDb().getSstFileList();
+        String tableName = table.getName();
+
+        // Only filter by column family if table name is available
+        if (tableName != null && !tableName.isEmpty()) {
+          byte[] tableNameBytes = tableName.getBytes(StandardCharsets.UTF_8);
+          for (LiveFileMetaData sstFile : sstFiles) {
+            // Filter SST files by column family to get bounds only for this 
specific table
+            if (Arrays.equals(sstFile.columnFamilyName(), tableNameBytes)) {
+              
keys.add(this.keyCodec.fromPersistedFormat(sstFile.smallestKey()));
+              
keys.add(this.keyCodec.fromPersistedFormat(sstFile.largestKey()));
+            }
+          }
+        }
+      }
+    } catch (Exception e) {
+      // If we can't get SST files (test environment, permissions, etc.),
+      // just use empty bounds and rely on fallback path
+      LOG.debug("Unable to retrieve SST file boundaries, will use fallback 
iteration: {}", e.getMessage());
+    }
+
+    if (startKey != null) {
+      keys.add(startKey);
+    }
+    if (endKey != null) {
+      keys.add(endKey);
+    }
+
+    return keys.stream().sorted().filter(Objects::nonNull)
+            .filter(key -> startKey == null || key.compareTo(startKey) >= 0)
+            .filter(key -> endKey == null || endKey.compareTo(key) >= 0)
+            .collect(Collectors.toList());
+  }
+
+  private void waitForQueueSize(Queue<Future<?>> futures, int expectedSize)
+          throws ExecutionException, InterruptedException {
+    while (!futures.isEmpty() && futures.size() > expectedSize) {
+      Future<?> f = futures.poll();
+      f.get();
+    }
+  }
+
+  public void performTaskOnTableVals(String taskName, K startKey, K endKey,
+      Function<Table.KeyValue<K, V>, Void> keyOperation) throws IOException, 
ExecutionException, InterruptedException {
+    List<K> bounds = getBounds(startKey, endKey);
+    
+    // Fallback for small tables (no SST files yet - data only in memtable)
+    if (bounds.size() < 2) {
+      try (TableIterator<K, ? extends Table.KeyValue<K, V>> iter = 
table.iterator()) {
+        if (startKey != null) {
+          iter.seek(startKey);
+        }
+        while (iter.hasNext()) {
+          Table.KeyValue<K, V> kv = iter.next();
+          if (endKey != null && kv.getKey().compareTo(endKey) > 0) {
+            break;
+          }
+          keyOperation.apply(kv);
+        }
+      }
+      return;
+    }
+    
+    Queue<Future<?>> iterFutures = new LinkedList<>();
+    Queue<Future<?>> workerFutures = new ConcurrentLinkedQueue<>();
+    AtomicLong keyCounter = new AtomicLong();
+    AtomicLong prevLogCounter = new AtomicLong();
+    for (int idx = 1; idx < bounds.size(); idx++) {
+      K beg = bounds.get(idx - 1);
+      K end = bounds.get(idx);
+      boolean inclusive = idx == bounds.size() - 1;
+      waitForQueueSize(iterFutures, maxIteratorTasks - 1);
+      iterFutures.add(iteratorExecutor.submit(() -> {
+        try (TableIterator<K, ? extends Table.KeyValue<K, V>> iter  = 
table.iterator()) {
+          iter.seek(beg);
+          while (iter.hasNext()) {
+            List<Table.KeyValue<K, V>> keyValues = new ArrayList<>();
+            boolean reachedEnd = false;
+            while (iter.hasNext()) {
+              Table.KeyValue<K, V> kv = iter.next();
+              K key = kv.getKey();
+              
+              // Check if key is within this segment's range
+              boolean withinBounds;
+              if (inclusive) {
+                // Last segment: include everything from beg onwards (or until 
endKey if specified)
+                withinBounds = (endKey == null || key.compareTo(endKey) <= 0);
+              } else {
+                // Middle segment: include keys in range [beg, end)
+                withinBounds = key.compareTo(end) < 0;
+              }
+              
+              if (withinBounds) {
+                keyValues.add(kv);
+              } else {
+                reachedEnd = true;
+                break;
+              }
+              if (keyValues.size() >= maxNumberOfVals) {
+                break;
+              }
+            }
+            if (!keyValues.isEmpty()) {
+              waitForQueueSize(workerFutures, maxWorkerTasks - 10);
+              workerFutures.add(valueExecutors.submit(() -> {
+                for (Table.KeyValue<K, V> kv : keyValues) {
+                  keyOperation.apply(kv);
+                }
+                keyCounter.addAndGet(keyValues.size());
+                if (keyCounter.get() - prevLogCounter.get() > 
logCountThreshold) {
+                  synchronized (keyCounter) {
+                    if (keyCounter.get() - prevLogCounter.get() > 
logCountThreshold) {
+                      long cnt = keyCounter.get();
+                      LOG.info("Iterated through {} keys while performing 
task: {}", keyCounter.get(), taskName);
+                      prevLogCounter.set(cnt);
+                    }
+                  }
+                }
+              }));
+            }
+            if (reachedEnd) {
+              break;
+            }
+          }
+        } catch (IOException | ExecutionException | InterruptedException e) {

Review Comment:
   Thanks, I have made the change.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java:
##########
@@ -134,31 +154,73 @@ public static boolean reprocessBucketLayout(BucketLayout 
bucketLayout,
                                               OMMetadataManager 
omMetadataManager,
                                               Map<FileSizeCountKey, Long> 
fileSizeCountMap,
                                               ReconFileMetadataManager 
reconFileMetadataManager,
-                                              String taskName) {
+                                              String taskName,
+                                              int maxIterators,
+                                              int maxWorkers,
+                                              int maxKeysInMemory) {
+    LOG.info("{}: Starting parallel iteration with {} iterators, {} workers 
for bucket layout {}",
+        taskName, maxIterators, maxWorkers, bucketLayout);
     Table<String, OmKeyInfo> omKeyInfoTable = 
omMetadataManager.getKeyTable(bucketLayout);
-    int totalKeysProcessed = 0;
+    long startTime = Time.monotonicNow();
     
-    try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>> 
keyIter =
-             omKeyInfoTable.iterator()) {
-      while (keyIter.hasNext()) {
-        Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
+    // Use fair lock to prevent write lock starvation when flushing
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+    // Flag to coordinate flush attempts - prevents all threads from queuing 
for write lock
+    AtomicBoolean isFlushingInProgress = new AtomicBoolean(false);
+    final int FLUSH_THRESHOLD = 100000;
+    
+    // Use parallel table iteration
+    Function<Table.KeyValue<String, OmKeyInfo>, Void> kvOperation = kv -> {
+      try {
+        lock.readLock().lock();
         handlePutKeyEvent(kv.getValue(), fileSizeCountMap);
-        totalKeysProcessed++;
-
-        // Flush to RocksDB periodically.
-        if (fileSizeCountMap.size() >= 100000) {
-          // For reprocess, we don't need to check existing values since table 
was truncated
-          LOG.debug("Flushing {} accumulated counts to RocksDB for {}", 
fileSizeCountMap.size(), taskName);
-          writeCountsToDB(fileSizeCountMap, reconFileMetadataManager);
-          fileSizeCountMap.clear();
+      } finally {
+        lock.readLock().unlock();
+      }
+      
+      // Only one thread should attempt flush to avoid blocking all workers
+      if (fileSizeCountMap.size() >= FLUSH_THRESHOLD &&
+          isFlushingInProgress.compareAndSet(false, true)) {
+        try {
+          lock.writeLock().lock();
+          try {
+            // Double-check after acquiring write lock
+            if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) {
+              LOG.debug("Flushing {} accumulated counts to RocksDB for {}", 
fileSizeCountMap.size(), taskName);
+              writeCountsToDB(fileSizeCountMap, reconFileMetadataManager);

Review Comment:
   Thanks, I have made the change.
   Using a shared lock for writing into the db
   this shared lock will be used across the two tasks OBS and FSO 
   
   



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