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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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++) {

Review Comment:
   When the pair is not satisfied due to filter, can it lead to missing data ? 
also pair is like i-1 index is lower bound and i is upper bound, but due to 
filter, this pair can get disturbed with start key and end key filter ?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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(() -> {

Review Comment:
   workerFutures will wait for the batch being queued. May have some delay in 
lesser number of splits. Otherway is,
   Blocking-workerQueue -- 
   Producer (multi-threaded, ie iterator-executor): kv is added as soon 
identified
   consumer: Set of thread waiting on queue to retrive and process
   End point: set flag when iterator is completed
   
   This might be simpler implementation to avoid intermediate wait.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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)

Review Comment:
   due to filter, pair of lowestKey and highestKey can get diverted, so this 
logic should work on pair, not on simple list with assumption,



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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);

Review Comment:
   Can adding startKey and endKey as additional iterator can result in extra 
iteration? as existing pair already added all possible cases. How is 
correctness of this logic ?



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