From 848d52bc21c61b3cc8a5c6a7f4bb20a1748a0505 Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 4 Nov 2025 10:44:23 +0530 Subject: [PATCH 01/49] HDDS-12607. Parallelize recon tasks to speed up OM rocksdb reading tasks. --- .../ozone/recon/ReconServerConfigKeys.java | 15 ++ .../recon/tasks/ContainerKeyMapperHelper.java | 98 +++++--- .../tasks/ContainerKeyMapperTaskFSO.java | 12 +- .../tasks/ContainerKeyMapperTaskOBS.java | 11 +- .../recon/tasks/FileSizeCountTaskHelper.java | 2 +- .../ozone/recon/tasks/NSSummaryTask.java | 23 +- .../tasks/NSSummaryTaskDbEventHandler.java | 74 +++--- .../recon/tasks/NSSummaryTaskWithFSO.java | 103 ++++++--- .../recon/tasks/NSSummaryTaskWithLegacy.java | 120 ++++++---- .../recon/tasks/NSSummaryTaskWithOBS.java | 100 +++++--- .../ozone/recon/tasks/OmTableInsightTask.java | 6 +- .../util/ParallelTableIteratorOperation.java | 215 ++++++++++++++++++ .../ozone/recon/tasks/util/package-info.java | 26 +++ .../recon/api/TestContainerEndpoint.java | 6 +- .../api/TestNSSummaryDiskUsageOrdering.java | 2 +- .../api/TestNSSummaryEndpointWithFSO.java | 2 +- .../api/TestNSSummaryEndpointWithLegacy.java | 2 +- ...TestNSSummaryEndpointWithOBSAndLegacy.java | 4 +- .../recon/api/TestOmDBInsightEndPoint.java | 6 +- .../recon/api/TestOpenKeysSearchEndpoint.java | 2 +- .../recon/tasks/TestNSSummaryTaskWithFSO.java | 2 +- .../tasks/TestNSSummaryTaskWithLegacy.java | 2 +- .../TestNSSummaryTaskWithLegacyOBSLayout.java | 2 +- .../recon/tasks/TestNSSummaryTaskWithOBS.java | 2 +- .../TestNSSummaryTreePrecomputeValues.java | 2 +- .../recon/tasks/TestOmTableInsightTask.java | 2 +- pom.xml | 2 +- 27 files changed, 634 insertions(+), 209 deletions(-) create mode 100644 hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java create mode 100644 hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/package-info.java diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java index d3f684238712..7c3bd6b8fcec 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java @@ -162,6 +162,21 @@ public final class ReconServerConfigKeys { public static final long OZONE_RECON_CONTAINER_KEY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT = 150 * 1000L; + public static final String + OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS = "ozone.recon.task.reprocess.max.iterators"; + + public static final int OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT = 5; + + public static final String + OZONE_RECON_TASK_REPROCESS_MAX_WORKERS = "ozone.recon.task.reprocess.max.workers"; + + public static final int OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT = 20; + + public static final String + OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY = "ozone.recon.task.reprocess.max.keys.in.memory"; + + public static final int OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT = 2000; + public static final String OZONE_RECON_SCM_SNAPSHOT_TASK_INTERVAL_DELAY = "ozone.recon.scm.snapshot.task.interval.delay"; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index e786aa282a8c..14ab2ff9cc71 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.recon.tasks; import java.io.IOException; +import java.io.UncheckedIOException; import java.time.Duration; import java.time.Instant; import java.util.ArrayList; @@ -28,7 +29,12 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; +import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -40,6 +46,7 @@ import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix; import org.apache.hadoop.ozone.recon.api.types.KeyPrefixContainer; import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager; +import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,13 +91,16 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, ReconContainerMetadataManager reconContainerMetadataManager, BucketLayout bucketLayout, String taskName, - long containerKeyFlushToDBMaxThreshold) { - long omKeyCount = 0; - Map containerKeyMap = new HashMap<>(); - Map containerKeyCountMap = new HashMap<>(); + long containerKeyFlushToDBMaxThreshold, + int maxIterators, + int maxWorkers, + int maxKeysInMemory) { + AtomicLong omKeyCount = new AtomicLong(0); + Map containerKeyMap = new ConcurrentHashMap<>(); + Map containerKeyCountMap = new ConcurrentHashMap<>(); try { - LOG.debug("Starting a 'reprocess' run for {}.", taskName); + LOG.info("Starting RocksDB Reprocess for {}", taskName); Instant start = Instant.now(); // Ensure the tables are truncated only once @@ -99,21 +109,38 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Get the appropriate table based on BucketLayout Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); - // Iterate through the table and process keys - try (TableIterator> keyIter = omKeyInfoTable.iterator()) { - while (keyIter.hasNext()) { - Table.KeyValue 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; + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + // Use parallel table iteration + Function, Void> kvOperation = kv -> { + try { + try { + lock.readLock().lock(); + handleKeyReprocess(kv.getKey(), kv.getValue(), containerKeyMap, containerKeyCountMap, + reconContainerMetadataManager); + } finally { + lock.readLock().unlock(); + } + omKeyCount.incrementAndGet(); + if (containerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { + try { + lock.writeLock().lock(); + if (!checkAndCallFlushToDB(containerKeyMap, containerKeyFlushToDBMaxThreshold, + reconContainerMetadataManager)) { + throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); + } + } finally { + lock.writeLock().unlock(); + } } + return null; + } catch (IOException e) { + throw new UncheckedIOException(e); } + }; + try (ParallelTableIteratorOperation keyIter = + new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, containerKeyFlushToDBMaxThreshold)) { + keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } // Final flush and commit @@ -125,17 +152,16 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, Instant end = Instant.now(); long durationMillis = Duration.between(start, end).toMillis(); double durationSeconds = (double) durationMillis / 1000.0; - LOG.debug("Completed 'reprocess' for {}. Processed {} keys in {} ms ({} seconds).", + LOG.info("Completed RocksDB Reprocess for {}. Processed {} keys in {} ms ({} seconds).", taskName, omKeyCount, durationMillis, durationSeconds); - - } catch (IOException ioEx) { - LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ioEx); + } catch (Exception ex) { + LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ex); return false; } return true; } - private static boolean checkAndCallFlushToDB(Map containerKeyMap, + private static synchronized boolean checkAndCallFlushToDB(Map containerKeyMap, long containerKeyFlushToDBMaxThreshold, ReconContainerMetadataManager reconContainerMetadataManager) { if (containerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { @@ -405,7 +431,7 @@ public static void handleKeyReprocess(String key, ReconContainerMetadataManager reconContainerMetadataManager) throws IOException { - long containerCountToIncrement = 0; + AtomicLong containerCountToIncrement = new AtomicLong(0); for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyInfo.getKeyLocationVersions()) { long keyVersion = omKeyLocationInfoGroup.getVersion(); @@ -418,23 +444,21 @@ public static void handleKeyReprocess(String key, // Save on writes. No need to save same container-key prefix mapping again. containerKeyMap.put(containerKeyPrefix, 1); - // Check if container already exists; if not, increment the count - if (!reconContainerMetadataManager.doesContainerExists(containerId) - && !containerKeyCountMap.containsKey(containerId)) { - containerCountToIncrement++; - } - - // Update the count of keys for the given containerID - long keyCount = containerKeyCountMap.getOrDefault(containerId, - reconContainerMetadataManager.getKeyCountForContainer(containerId)); - - containerKeyCountMap.put(containerId, keyCount + 1); + // if it exists, update the count of keys for the given containerID + // else, increment the count of containers and initialize keyCount + containerKeyCountMap.compute(containerId, (k, v) -> { + if (v == null) { + containerCountToIncrement.incrementAndGet(); + return 1L; + } + return v + 1L; + }); } } } - if (containerCountToIncrement > 0) { - reconContainerMetadataManager.incrementContainerCountBy(containerCountToIncrement); + if (containerCountToIncrement.get() > 0) { + reconContainerMetadataManager.incrementContainerCountBy(containerCountToIncrement.get()); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java index 60499ac33c8a..9b3374248bd3 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java @@ -55,9 +55,19 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { long containerKeyFlushToDBMaxThreshold = ozoneConfiguration.getLong( ReconServerConfigKeys.OZONE_RECON_CONTAINER_KEY_FLUSH_TO_DB_MAX_THRESHOLD, ReconServerConfigKeys.OZONE_RECON_CONTAINER_KEY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); + int maxKeysInMemory = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT); + int maxIterators = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); + int maxWorkers = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); boolean result = ContainerKeyMapperHelper.reprocess( omMetadataManager, reconContainerMetadataManager, - BucketLayout.FILE_SYSTEM_OPTIMIZED, getTaskName(), containerKeyFlushToDBMaxThreshold); + BucketLayout.FILE_SYSTEM_OPTIMIZED, getTaskName(), containerKeyFlushToDBMaxThreshold, + maxIterators, maxWorkers, maxKeysInMemory); return buildTaskResult(result); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskOBS.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskOBS.java index e69601040d60..47c19d15b9ed 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskOBS.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskOBS.java @@ -55,9 +55,18 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { long containerKeyFlushToDBMaxThreshold = ozoneConfiguration.getLong( ReconServerConfigKeys.OZONE_RECON_CONTAINER_KEY_FLUSH_TO_DB_MAX_THRESHOLD, ReconServerConfigKeys.OZONE_RECON_CONTAINER_KEY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); + int maxKeysInMemory = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT); + int maxIterators = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); + int maxWorkers = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); boolean result = ContainerKeyMapperHelper.reprocess( omMetadataManager, reconContainerMetadataManager, BucketLayout.OBJECT_STORE, getTaskName(), - containerKeyFlushToDBMaxThreshold); + containerKeyFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); return buildTaskResult(result); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index dfc579d8156a..4eef7ee6916c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -122,7 +122,7 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); long endTime = Time.monotonicNow(); - LOG.info("{} completed RocksDB Reprocess in {} ms.", taskName, (endTime - startTime)); + LOG.info("Completed RocksDB Reprocess for {} in {}", taskName, (endTime - startTime)); return buildTaskResult(taskName, true); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index 6b4a5cce0e71..3856af5967b3 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java @@ -41,8 +41,10 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,15 +106,25 @@ public NSSummaryTask(ReconNamespaceSummaryManager long nsSummaryFlushToDBMaxThreshold = ozoneConfiguration.getLong( OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); + int maxKeysInMemory = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT); + int maxIterators = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); + int maxWorkers = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, - nsSummaryFlushToDBMaxThreshold); + nsSummaryFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, reconOMMetadataManager, - ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( - reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold); + reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, + maxIterators, maxWorkers, maxKeysInMemory); } @Override @@ -216,7 +228,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { return buildTaskResult(false); } - LOG.info("Starting NSSummary tree reprocess with unified control..."); + LOG.info("Starting RocksDB Reprocess for {}", getTaskName()); long startTime = System.nanoTime(); // Record start time try { @@ -234,6 +246,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { protected TaskResult executeReprocess(OMMetadataManager omMetadataManager, long startTime) { // Initialize a list of tasks to run in parallel Collection> tasks = new ArrayList<>(); + LOG.info("Starting RocksDB Reprocess for {}", getTaskName()); try { // reinit Recon RocksDB's namespace CF. @@ -305,7 +318,7 @@ protected TaskResult executeReprocess(OMMetadataManager omMetadataManager, long // Reset state to IDLE on successful completion if (success) { REBUILD_STATE.set(RebuildState.IDLE); - LOG.info("NSSummary tree reprocess completed successfully with unified control."); + LOG.info("Completed RocksDB Reprocess for {} in {}", getTaskName(), (endTime - startTime)); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java index e83fc64ad4cc..85556ac379e0 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java @@ -87,34 +87,35 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); // Try to get the NSSummary from our local map that maps NSSummaries to IDs - NSSummary nsSummary = nsSummaryMap.get(parentObjectId); - if (nsSummary == null) { - // If we don't have it in this batch we try to get it from the DB - nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); - } - if (nsSummary == null) { - // If we don't have it locally and in the DB we create a new instance - // as this is a new ID - nsSummary = new NSSummary(); - } - int[] fileBucket = nsSummary.getFileSizeBucket(); - - // Update immediate parent's totals (includes all descendant files) - nsSummary.setNumOfFiles(nsSummary.getNumOfFiles() + 1); - nsSummary.setSizeOfFiles(nsSummary.getSizeOfFiles() + keyInfo.getDataSize()); - // Before arithmetic operations, check for sentinel value - long currentReplSize = nsSummary.getReplicatedSizeOfFiles(); - if (currentReplSize < 0) { - // Old data, initialize to 0 before first use - currentReplSize = 0; - nsSummary.setReplicatedSizeOfFiles(0); - } - nsSummary.setReplicatedSizeOfFiles(currentReplSize + keyInfo.getReplicatedSize()); - int binIndex = ReconUtils.getFileSizeBinIndex(keyInfo.getDataSize()); + nsSummaryMap.compute(parentObjectId, (k, v) -> { + if (v == null) { + try { + v = reconNamespaceSummaryManager.getNSSummary(parentObjectId); + if (v == null) { + v = new NSSummary(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + NSSummary nsSummary = v; + int[] fileBucket = nsSummary.getFileSizeBucket(); + nsSummary.setNumOfFiles(nsSummary.getNumOfFiles() + 1); + nsSummary.setSizeOfFiles(nsSummary.getSizeOfFiles() + keyInfo.getDataSize()); + // Before arithmetic operations, check for sentinel value + long currentReplSize = nsSummary.getReplicatedSizeOfFiles(); + if (currentReplSize < 0) { + // Old data, initialize to 0 before first use + currentReplSize = 0; + nsSummary.setReplicatedSizeOfFiles(0); + } + nsSummary.setReplicatedSizeOfFiles(currentReplSize + keyInfo.getReplicatedSize()); + int binIndex = ReconUtils.getFileSizeBinIndex(keyInfo.getDataSize()); - ++fileBucket[binIndex]; - nsSummary.setFileSizeBucket(fileBucket); - nsSummaryMap.put(parentObjectId, nsSummary); + ++fileBucket[binIndex]; + nsSummary.setFileSizeBucket(fileBucket); + return nsSummary; + }); // Propagate upwards to all parents in the parent chain propagateSizeUpwards(parentObjectId, keyInfo.getDataSize(), keyInfo.getReplicatedSize(), 1, nsSummaryMap); @@ -253,14 +254,17 @@ protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, } } - protected boolean flushAndCommitNSToDB(Map nsSummaryMap) { - try { - updateNSSummariesToDB(nsSummaryMap, Collections.emptyList()); - } catch (IOException e) { - LOG.error("Unable to write Namespace Summary data in Recon DB.", e); - return false; - } finally { - nsSummaryMap.clear(); + protected synchronized boolean flushAndCommitNSToDB(Map nsSummaryMap, + long nsSummaryFlushToDBMaxThreshold) { + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + try { + updateNSSummariesToDB(nsSummaryMap, Collections.emptyList()); + } catch (IOException e) { + LOG.error("Unable to write Namespace Summary data in Recon DB.", e); + return false; + } finally { + nsSummaryMap.clear(); + } } return true; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java index 0e7f41c16b56..be0696832dd3 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.FILE_TABLE; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -29,8 +30,12 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -40,6 +45,7 @@ import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,14 +58,23 @@ public class NSSummaryTaskWithFSO extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithFSO.class); private final long nsSummaryFlushToDBMaxThreshold; + private final int maxKeysInMemory; + private final int maxIterators; + private final int maxWorkers; public NSSummaryTaskWithFSO(ReconNamespaceSummaryManager reconNamespaceSummaryManager, ReconOMMetadataManager reconOMMetadataManager, - long nsSummaryFlushToDBMaxThreshold) { + long nsSummaryFlushToDBMaxThreshold, + int maxIterators, + int maxWorkers, + int maxKeysInMemory) { super(reconNamespaceSummaryManager, reconOMMetadataManager); + this.maxIterators = maxIterators; + this.maxWorkers = maxWorkers; + this.maxKeysInMemory = maxKeysInMemory; this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; } @@ -222,51 +237,83 @@ private void handleUpdateOnFileTable(OMDBUpdateEvent nsSummaryMap = new HashMap<>(); + Map nsSummaryMap = new ConcurrentHashMap<>(); try { - Table dirTable = - omMetadataManager.getDirectoryTable(); - try (TableIterator> - dirTableIter = dirTable.iterator()) { - while (dirTableIter.hasNext()) { - Table.KeyValue kv = dirTableIter.next(); - OmDirectoryInfo directoryInfo = kv.getValue(); - handlePutDirEvent(directoryInfo, nsSummaryMap); - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + Table dirTable = omMetadataManager.getDirectoryTable(); + try (ParallelTableIteratorOperation + dirTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, dirTable, + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + Function, Void> dirOperation = kv -> { + try { + OmDirectoryInfo directoryInfo = kv.getValue(); + try { + lock.readLock().lock(); + handlePutDirEvent(directoryInfo, nsSummaryMap); + } finally { + lock.readLock().unlock(); + } + + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + try { + lock.writeLock().lock(); + if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { + throw new IOException("Unable to flush NSSummaryMap to DB"); + } + } finally { + lock.writeLock().unlock(); + } } + } catch (IOException e) { + throw new UncheckedIOException(e); } - } + return null; + }; + dirTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, dirOperation); } // Get fileTable used by FSO Table keyTable = omMetadataManager.getFileTable(); - try (TableIterator> - keyTableIter = keyTable.iterator()) { - while (keyTableIter.hasNext()) { - Table.KeyValue kv = keyTableIter.next(); - OmKeyInfo keyInfo = kv.getValue(); - handlePutKeyEvent(keyInfo, nsSummaryMap); - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + try (ParallelTableIteratorOperation + keyTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + Function, Void> keyOperation = kv -> { + try { + OmKeyInfo keyInfo = kv.getValue(); + try { + lock.readLock().lock(); + handlePutKeyEvent(keyInfo, nsSummaryMap); + } finally { + lock.readLock().unlock(); + } + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + try { + lock.writeLock().lock(); + if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { + throw new IOException("Unable to flush NSSummaryMap to DB"); + } + } finally { + lock.writeLock().unlock(); + } } + } catch (IOException e) { + throw new RuntimeException(e); } - } + return null; + }; + keyTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, keyOperation); } - } catch (IOException ioEx) { + } catch (Exception ex) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", - ioEx); + ex); return false; } // flush and commit left out keys at end - if (!flushAndCommitNSToDB(nsSummaryMap)) { + if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { LOG.info("flushAndCommitNSToDB failed during reprocessWithFSO."); return false; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java index 6ad94797603b..91e8345df4bf 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java @@ -25,9 +25,13 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -40,6 +44,7 @@ import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,6 +60,9 @@ public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler { private final boolean enableFileSystemPaths; private final long nsSummaryFlushToDBMaxThreshold; + private final int maxKeysInMemory; + private final int maxIterators; + private final int maxWorkers; public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager reconNamespaceSummaryManager, @@ -62,7 +70,10 @@ public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager reconOMMetadataManager, OzoneConfiguration ozoneConfiguration, - long nsSummaryFlushToDBMaxThreshold) { + long nsSummaryFlushToDBMaxThreshold, + int maxIterators, + int maxWorkers, + int maxKeysInMemory) { super(reconNamespaceSummaryManager, reconOMMetadataManager); // true if FileSystemPaths enabled @@ -70,6 +81,9 @@ public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager .getBoolean(OmConfig.Keys.ENABLE_FILESYSTEM_PATHS, OmConfig.Defaults.ENABLE_FILESYSTEM_PATHS); this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; + this.maxIterators = maxIterators; + this.maxWorkers = maxWorkers; + this.maxKeysInMemory = maxKeysInMemory; } public Pair processWithLegacy(OMUpdateEventBatch events, @@ -132,7 +146,7 @@ public Pair processWithLegacy(OMUpdateEventBatch events, return new ImmutablePair<>(seekPos, false); } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap)) { + if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { return new ImmutablePair<>(seekPos, false); } seekPos = eventCounter + 1; @@ -140,7 +154,7 @@ public Pair processWithLegacy(OMUpdateEventBatch events, } // flush and commit left out entries at end - if (!flushAndCommitNSToDB(nsSummaryMap)) { + if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { return new ImmutablePair<>(seekPos, false); } @@ -255,63 +269,79 @@ private void processWithObjectStoreLayout(OmKeyInfo updatedKeyInfo, } public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) { - Map nsSummaryMap = new HashMap<>(); + Map nsSummaryMap = new ConcurrentHashMap<>(); try { Table keyTable = omMetadataManager.getKeyTable(LEGACY_BUCKET_LAYOUT); + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + try (ParallelTableIteratorOperation + keyTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, StringCodec.get(), + maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + + Function, Void> keyOperation = kv -> { + try { + OmKeyInfo keyInfo = kv.getValue(); + + // KeyTable entries belong to both Legacy and OBS buckets. + // Check bucket layout and if it's OBS + // continue to the next iteration. + if (!isBucketLayoutValid((ReconOMMetadataManager) omMetadataManager, + keyInfo)) { + return null; + } - try (TableIterator> - keyTableIter = keyTable.iterator()) { - - while (keyTableIter.hasNext()) { - Table.KeyValue kv = keyTableIter.next(); - OmKeyInfo keyInfo = kv.getValue(); - - // KeyTable entries belong to both Legacy and OBS buckets. - // Check bucket layout and if it's OBS - // continue to the next iteration. - if (!isBucketLayoutValid((ReconOMMetadataManager) omMetadataManager, - keyInfo)) { - continue; - } - - if (enableFileSystemPaths) { - // The LEGACY bucket is a file system bucket. - setKeyParentID(keyInfo); - - if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) { - OmDirectoryInfo directoryInfo = - new OmDirectoryInfo.Builder() - .setName(keyInfo.getKeyName()) - .setObjectID(keyInfo.getObjectID()) - .setParentObjectID(keyInfo.getParentObjectID()) - .build(); - handlePutDirEvent(directoryInfo, nsSummaryMap); - } else { - handlePutKeyEvent(keyInfo, nsSummaryMap); + try { + lock.readLock().lock(); + if (enableFileSystemPaths) { + // The LEGACY bucket is a file system bucket. + setKeyParentID(keyInfo); + + if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) { + OmDirectoryInfo directoryInfo = + new OmDirectoryInfo.Builder() + .setName(keyInfo.getKeyName()) + .setObjectID(keyInfo.getObjectID()) + .setParentObjectID(keyInfo.getParentObjectID()) + .build(); + handlePutDirEvent(directoryInfo, nsSummaryMap); + } else { + handlePutKeyEvent(keyInfo, nsSummaryMap); + } + } else { + // The LEGACY bucket is an object store bucket. + setParentBucketId(keyInfo); + handlePutKeyEvent(keyInfo, nsSummaryMap); + } + } finally { + lock.readLock().unlock(); } - } else { - // The LEGACY bucket is an object store bucket. - setParentBucketId(keyInfo); - handlePutKeyEvent(keyInfo, nsSummaryMap); - } - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + try { + lock.writeLock().lock(); + if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { + throw new IOException(); + } + } finally { + lock.writeLock().unlock(); + } } + } catch (IOException e) { + throw new RuntimeException(e); } - } + return null; + }; + keyTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, keyOperation); } - } catch (IOException ioEx) { + } catch (Exception ex) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", - ioEx); + ex); nsSummaryMap.clear(); return false; } // flush and commit left out entries at end - if (!flushAndCommitNSToDB(nsSummaryMap)) { + if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { return false; } LOG.debug("Completed a reprocess run of NSSummaryTaskWithLegacy"); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java index e1c6f4542c46..b61880233d90 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java @@ -23,8 +23,12 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -35,6 +39,7 @@ import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,64 +54,87 @@ public class NSSummaryTaskWithOBS extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithOBS.class); private final long nsSummaryFlushToDBMaxThreshold; + private final int maxKeysInMemory; + private final int maxIterators; + private final int maxWorkers; public NSSummaryTaskWithOBS( ReconNamespaceSummaryManager reconNamespaceSummaryManager, ReconOMMetadataManager reconOMMetadataManager, - long nsSummaryFlushToDBMaxThreshold) { + long nsSummaryFlushToDBMaxThreshold, int maxIterators, int maxWorkers, int maxKeysInMemory) { super(reconNamespaceSummaryManager, reconOMMetadataManager); this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; + this.maxIterators = maxIterators; + this.maxWorkers = maxWorkers; + this.maxKeysInMemory = maxKeysInMemory; } public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { - Map nsSummaryMap = new HashMap<>(); - + Map nsSummaryMap = new ConcurrentHashMap<>(); + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); try { Table keyTable = omMetadataManager.getKeyTable(BUCKET_LAYOUT); - try (TableIterator> - keyTableIter = keyTable.iterator()) { - - while (keyTableIter.hasNext()) { - Table.KeyValue kv = keyTableIter.next(); - OmKeyInfo keyInfo = kv.getValue(); - - // KeyTable entries belong to both Legacy and OBS buckets. - // Check bucket layout and if it's anything other than OBS, - // continue to the next iteration. - String volumeName = keyInfo.getVolumeName(); - String bucketName = keyInfo.getBucketName(); - String bucketDBKey = omMetadataManager - .getBucketKey(volumeName, bucketName); - // Get bucket info from bucket table - OmBucketInfo omBucketInfo = omMetadataManager - .getBucketTable().getSkipCache(bucketDBKey); - - if (omBucketInfo.getBucketLayout() != BUCKET_LAYOUT) { - continue; - } + try (ParallelTableIteratorOperation + keyTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, StringCodec.get(), + maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + + Function, Void> keyOperation = kv -> { + try { + OmKeyInfo keyInfo = kv.getValue(); + + // KeyTable entries belong to both Legacy and OBS buckets. + // Check bucket layout and if it's anything other than OBS, + // continue to the next iteration. + String volumeName = keyInfo.getVolumeName(); + String bucketName = keyInfo.getBucketName(); + String bucketDBKey = omMetadataManager + .getBucketKey(volumeName, bucketName); + // Get bucket info from bucket table + OmBucketInfo omBucketInfo = omMetadataManager + .getBucketTable().getSkipCache(bucketDBKey); + + if (omBucketInfo.getBucketLayout() != BUCKET_LAYOUT) { + return null; + } - setKeyParentID(keyInfo); + setKeyParentID(keyInfo); - handlePutKeyEvent(keyInfo, nsSummaryMap); - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + try { + lock.readLock().lock(); + handlePutKeyEvent(keyInfo, nsSummaryMap); + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + lock.readLock().unlock(); + } + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + try { + lock.writeLock().lock(); + if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { + throw new IOException("Failed to commit nsSummaryMap"); + } + } finally { + lock.writeLock().unlock(); + } } + } catch (IOException e) { + throw new RuntimeException(e); } - } + return null; + }; + keyTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, keyOperation); } - } catch (IOException ioEx) { - LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", - ioEx); + } catch (Exception exception) { + LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", exception); nsSummaryMap.clear(); return false; } // flush and commit left out entries at end - if (!flushAndCommitNSToDB(nsSummaryMap)) { + if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { return false; } LOG.debug("Completed a reprocess run of NSSummaryTaskWithOBS"); @@ -196,7 +224,7 @@ public Pair processWithOBS(OMUpdateEventBatch events, LOG.debug("Skipping DB update event: {}", action); } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap)) { + if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { return new ImmutablePair<>(seekPos, false); } seekPos = eventCounter + 1; @@ -210,7 +238,7 @@ public Pair processWithOBS(OMUpdateEventBatch events, } // Flush and commit left-out entries at the end - if (!flushAndCommitNSToDB(nsSummaryMap)) { + if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { return new ImmutablePair<>(seekPos, false); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 0ea225e12fbb..cf81b4e35b59 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java @@ -108,6 +108,9 @@ public void init() { */ @Override public TaskResult reprocess(OMMetadataManager omMetadataManager) { + LOG.info("Starting RocksDB Reprocess for {}", getTaskName()); + long startTime = Time.monotonicNow(); + init(); for (String tableName : tables) { Table table = omMetadataManager.getTable(tableName); @@ -142,8 +145,9 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { if (!replicatedSizeMap.isEmpty()) { writeDataToDB(replicatedSizeMap); } + long endTime = Time.monotonicNow(); - LOG.debug("Completed a 'reprocess' run of OmTableInsightTask."); + LOG.info("Completed RocksDB Reprocess for {} in {}", getTaskName(), (endTime - startTime)); return buildTaskResult(true); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java new file mode 100644 index 000000000000..0640d4ad30b0 --- /dev/null +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -0,0 +1,215 @@ +/* + * 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, V> implements Closeable { + private final Table table; + private final Codec 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 table, Codec 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)); + this.valueExecutors = new ThreadPoolExecutor(workerCount, workerCount, 1, TimeUnit.MINUTES, + new ArrayBlockingQueue<>(workerCount * 2)); + this.maxNumberOfVals = Math.max(10, maxNumberOfValsInMemory / (workerCount)); + this.logCountThreshold = logThreshold; + } + + + private List getBounds(K startKey, K endKey) throws IOException { + RDBStore store = (RDBStore) this.metadataManager.getStore(); + List sstFiles = store.getDb().getSstFileList(); + Set keys = new HashSet<>(); + String tableName = table.getName(); + 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())); + } + } + 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> 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, Void> keyOperation) throws IOException, ExecutionException, InterruptedException { + List bounds = getBounds(startKey, endKey); + + // Fallback for small tables (no SST files yet - data only in memtable) + if (bounds.size() < 2) { + try (TableIterator> iter = table.iterator()) { + if (startKey != null) { + iter.seek(startKey); + } + while (iter.hasNext()) { + Table.KeyValue kv = iter.next(); + if (endKey != null && kv.getKey().compareTo(endKey) > 0) { + break; + } + keyOperation.apply(kv); + } + } + return; + } + + Queue> iterFutures = new LinkedList<>(); + Queue> 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> iter = table.iterator()) { + iter.seek(beg); + while (iter.hasNext()) { + List> keyValues = new ArrayList<>(); + boolean reachedEnd = false; + while (iter.hasNext()) { + Table.KeyValue 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 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) { + throw new RuntimeException(e); + } + })); + } + waitForQueueSize(iterFutures, 0); + waitForQueueSize(workerFutures, 0); + } + + @Override + public void close() throws IOException { + iteratorExecutor.shutdown(); + valueExecutors.shutdown(); + } +} + + + diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/package-info.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/package-info.java new file mode 100644 index 000000000000..f8ec57de2f2f --- /dev/null +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/package-info.java @@ -0,0 +1,26 @@ +/* + * 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 to define utility classes for tasks. + */ +package org.apache.hadoop.ozone.recon.tasks.util; + + + + + diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index 4794ecf1f309..b41c4feaae1d 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -481,7 +481,7 @@ public void testGetKeysForContainer() throws IOException { setUpFSOData(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); // Reprocess the container key mapper to ensure the latest mapping is used reprocessContainerKeyMapper(); @@ -569,7 +569,7 @@ public void testGetKeysForContainerWithPrevKey() throws IOException { reprocessContainerKeyMapper(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); response = containerEndpoint.getKeysForContainer(20L, -1, "/0/1/2/file7"); @@ -1826,7 +1826,7 @@ public void testDuplicateFSOKeysForContainerEndpoint() throws IOException { setUpDuplicateFSOFileKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); // Reprocess the container key mappings so that the new keys are loaded. reprocessContainerKeyMapper(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java index 2d375b6fe866..c9f6c5dbf182 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java @@ -105,7 +105,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java index 557115150d28..90494ef02c4f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java @@ -434,7 +434,7 @@ public void setUp() throws Exception { setUpMultiBlockReplicatedKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java index 8844b96c0226..7546e649afad 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java @@ -435,7 +435,7 @@ public void setUp() throws Exception { setUpMultiBlockReplicatedKeys(); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, - reconOMMetadataManager, conf, 10); + reconOMMetadataManager, conf, 10, 1, 1, 10); nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java index eefc11940605..8392a6c51187 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java @@ -371,11 +371,11 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, - reconOMMetadataManager, conf, 10); + reconOMMetadataManager, conf, 10, 1, 1, 10); nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java index c83501bb7a0d..0729f8c18b7c 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java @@ -328,13 +328,13 @@ public void setUp() throws Exception { setUpOmData(); NSSummaryTaskWithLegacy nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration, 10); + reconOMMetadataManager, ozoneConfiguration, 10, 1, 1, 10); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); NSSummaryTaskWithFSO nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); reconNamespaceSummaryManager.clearNSSummaryTable(); nSSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java index b59aaa3c8b8f..c53b1876c743 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java @@ -113,7 +113,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10, 1, 1, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java index e93f9176c142..292dc76e9775 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java @@ -81,7 +81,7 @@ void setUp(@TempDir File tmpDir) throws Exception { nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - threshold); + threshold, 1, 1, 10); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java index ecd886f276b6..3ef325a8bb57 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java @@ -65,7 +65,7 @@ void setUp(@TempDir File tmpDir) throws Exception { long threshold = getOmConfiguration().getLong(OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, 10); nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - getOmConfiguration(), threshold); + getOmConfiguration(), threshold, 1, 1, 10); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java index 795e5d1da334..cc0254622c4f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java @@ -61,7 +61,7 @@ void setUp(@TempDir File tmpDir) throws Exception { nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(getReconNamespaceSummaryManager(), getReconOMMetadataManager(), getOzoneConfiguration(), - threshold); + threshold, 1, 1, 10); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java index 8d44633ba7ff..bf0c0c22775d 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java @@ -61,7 +61,7 @@ false, getBucketLayout(), OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); nSSummaryTaskWithOBS = new NSSummaryTaskWithOBS(getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - threshold); + threshold, 1, 1, 10); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java index 3dd691110611..14cb28f7761a 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java @@ -112,7 +112,7 @@ void setUp(@TempDir File tmpDir) throws Exception { nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - threshold); + threshold, 1, 1, 10); // Populate a custom complex tree structure populateComplexTree(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java index 659ece4b3987..27c5242087c6 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java @@ -144,7 +144,7 @@ private void initializeInjector() throws IOException { omTableInsightTask = new OmTableInsightTask( reconGlobalStatsManager, reconOMMetadataManager); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( - reconNamespaceSummaryManager, reconOMMetadataManager, 10); + reconNamespaceSummaryManager, reconOMMetadataManager, 10, 1, 1, 10); dslContext = getDslContext(); omTableInsightTask.setTables(omTableInsightTask.getTaskTables()); diff --git a/pom.xml b/pom.xml index db7fa29ddc02..f1e5bbf412df 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ 1.9.7 3.27.6 1.12.788 - 2.37.3 + 2.36.2 0.8.0.RELEASE 1.82 3.6.1 From b9100a6de6f5170c62e8be7d3679751848ac0a29 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 5 Nov 2025 13:58:28 +0530 Subject: [PATCH 02/49] Reverted the changes for NSSummaryTask --- .../ozone/recon/tasks/NSSummaryTask.java | 17 +- .../tasks/NSSummaryTaskDbEventHandler.java | 50 +++--- .../recon/tasks/NSSummaryTaskWithFSO.java | 147 ++++++------------ .../recon/tasks/NSSummaryTaskWithLegacy.java | 62 ++------ .../recon/tasks/NSSummaryTaskWithOBS.java | 62 ++------ .../recon/api/TestContainerEndpoint.java | 6 +- .../api/TestNSSummaryDiskUsageOrdering.java | 2 +- .../api/TestNSSummaryEndpointWithFSO.java | 2 +- .../api/TestNSSummaryEndpointWithLegacy.java | 2 +- ...TestNSSummaryEndpointWithOBSAndLegacy.java | 4 +- .../recon/api/TestOmDBInsightEndPoint.java | 6 +- .../recon/api/TestOpenKeysSearchEndpoint.java | 2 +- .../recon/tasks/TestNSSummaryTaskWithFSO.java | 2 +- .../tasks/TestNSSummaryTaskWithLegacy.java | 2 +- .../TestNSSummaryTaskWithLegacyOBSLayout.java | 2 +- .../recon/tasks/TestNSSummaryTaskWithOBS.java | 2 +- .../TestNSSummaryTreePrecomputeValues.java | 2 +- .../recon/tasks/TestOmTableInsightTask.java | 2 +- 18 files changed, 127 insertions(+), 247 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index 3856af5967b3..ebd0f6f67e12 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.ozone.om.OMMetadataManager; -import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; import org.apache.hadoop.util.Time; @@ -106,25 +105,15 @@ public NSSummaryTask(ReconNamespaceSummaryManager long nsSummaryFlushToDBMaxThreshold = ozoneConfiguration.getLong( OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); - int maxKeysInMemory = ozoneConfiguration.getInt( - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY, - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT); - int maxIterators = ozoneConfiguration.getInt( - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); - int maxWorkers = ozoneConfiguration.getInt( - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, - nsSummaryFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); + nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, reconOMMetadataManager, - ozoneConfiguration, nsSummaryFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( - reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, - maxIterators, maxWorkers, maxKeysInMemory); + reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold); } @Override diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java index 85556ac379e0..c28054ccd169 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java @@ -87,35 +87,34 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); // Try to get the NSSummary from our local map that maps NSSummaries to IDs - nsSummaryMap.compute(parentObjectId, (k, v) -> { - if (v == null) { - try { - v = reconNamespaceSummaryManager.getNSSummary(parentObjectId); - if (v == null) { - v = new NSSummary(); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - } - NSSummary nsSummary = v; + NSSummary nsSummary = nsSummaryMap.get(parentObjectId); + if (nsSummary == null) { + // If we don't have it in this batch we try to get it from the DB + nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); + } + if (nsSummary == null) { + // If we don't have it locally and in the DB we create a new instance + // as this is a new ID + nsSummary = new NSSummary(); + } int[] fileBucket = nsSummary.getFileSizeBucket(); + + // Update immediate parent's totals (includes all descendant files) nsSummary.setNumOfFiles(nsSummary.getNumOfFiles() + 1); nsSummary.setSizeOfFiles(nsSummary.getSizeOfFiles() + keyInfo.getDataSize()); - // Before arithmetic operations, check for sentinel value - long currentReplSize = nsSummary.getReplicatedSizeOfFiles(); - if (currentReplSize < 0) { - // Old data, initialize to 0 before first use - currentReplSize = 0; - nsSummary.setReplicatedSizeOfFiles(0); - } - nsSummary.setReplicatedSizeOfFiles(currentReplSize + keyInfo.getReplicatedSize()); + // Before arithmetic operations, check for sentinel value + long currentReplSize = nsSummary.getReplicatedSizeOfFiles(); + if (currentReplSize < 0) { + // Old data, initialize to 0 before first use + currentReplSize = 0; + nsSummary.setReplicatedSizeOfFiles(0); + } + nsSummary.setReplicatedSizeOfFiles(currentReplSize + keyInfo.getReplicatedSize()); int binIndex = ReconUtils.getFileSizeBinIndex(keyInfo.getDataSize()); ++fileBucket[binIndex]; nsSummary.setFileSizeBucket(fileBucket); - return nsSummary; - }); + nsSummaryMap.put(parentObjectId, nsSummary); // Propagate upwards to all parents in the parent chain propagateSizeUpwards(parentObjectId, keyInfo.getDataSize(), keyInfo.getReplicatedSize(), 1, nsSummaryMap); @@ -254,18 +253,15 @@ protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, } } - protected synchronized boolean flushAndCommitNSToDB(Map nsSummaryMap, - long nsSummaryFlushToDBMaxThreshold) { - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + protected boolean flushAndCommitNSToDB(Map nsSummaryMap) { try { - updateNSSummariesToDB(nsSummaryMap, Collections.emptyList()); + updateNSSummariesToDB(nsSummaryMap, Collections.emptyList()); } catch (IOException e) { LOG.error("Unable to write Namespace Summary data in Recon DB.", e); return false; } finally { nsSummaryMap.clear(); } - } return true; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java index be0696832dd3..b0d38419ba53 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java @@ -22,7 +22,6 @@ import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.FILE_TABLE; import java.io.IOException; -import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -30,12 +29,8 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Function; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -45,7 +40,6 @@ import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; -import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,23 +52,14 @@ public class NSSummaryTaskWithFSO extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithFSO.class); private final long nsSummaryFlushToDBMaxThreshold; - private final int maxKeysInMemory; - private final int maxIterators; - private final int maxWorkers; public NSSummaryTaskWithFSO(ReconNamespaceSummaryManager reconNamespaceSummaryManager, ReconOMMetadataManager reconOMMetadataManager, - long nsSummaryFlushToDBMaxThreshold, - int maxIterators, - int maxWorkers, - int maxKeysInMemory) { + long nsSummaryFlushToDBMaxThreshold) { super(reconNamespaceSummaryManager, reconOMMetadataManager); - this.maxIterators = maxIterators; - this.maxWorkers = maxWorkers; - this.maxKeysInMemory = maxKeysInMemory; this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; } @@ -97,7 +82,7 @@ public Pair processWithFSO(OMUpdateEventBatch events, final Collection objectIdsToBeDeleted = Collections.synchronizedList(new ArrayList<>()); while (eventIterator.hasNext()) { OMDBUpdateEvent omdbUpdateEvent = eventIterator.next(); + WithParentObjectId> omdbUpdateEvent = eventIterator.next(); OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction(); eventCounter++; @@ -149,35 +134,35 @@ public Pair processWithFSO(OMUpdateEventBatch events, private void handleUpdateOnDirTable(OMDBUpdateEvent omdbUpdateEvent, OMDBUpdateEvent.OMDBUpdateAction action, Map nsSummaryMap) throws IOException { - OMDBUpdateEvent dirTableUpdateEvent = - (OMDBUpdateEvent) omdbUpdateEvent; - OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue(); - OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue(); - - switch (action) { - case PUT: - handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); - break; - - case DELETE: - handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap); - break; - - case UPDATE: - if (oldDirectoryInfo != null) { - // delete first, then put - handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap); - } else { + OMDBUpdateEvent dirTableUpdateEvent = + (OMDBUpdateEvent) omdbUpdateEvent; + OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue(); + OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue(); + + switch (action) { + case PUT: + handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); + break; + + case DELETE: + handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap); + break; + + case UPDATE: + if (oldDirectoryInfo != null) { + // delete first, then put + handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap); + } else { LOG.warn("Update event does not have the old dirInfo for {}.", dirTableUpdateEvent.getKey()); - } - handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); - break; + } + handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); + break; - default: - LOG.debug("Skipping DB update event : {}", - omdbUpdateEvent.getAction()); - } - } + default: + LOG.debug("Skipping DB update event : {}", + omdbUpdateEvent.getAction()); + } + } private void handleUpdateOnDeletedDirTable(OMDBUpdateEvent omdbUpdateEvent, OMDBUpdateEvent.OMDBUpdateAction action, Map nsSummaryMap, @@ -237,83 +222,51 @@ private void handleUpdateOnFileTable(OMDBUpdateEvent nsSummaryMap = new ConcurrentHashMap<>(); + Map nsSummaryMap = new HashMap<>(); try { - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - Table dirTable = omMetadataManager.getDirectoryTable(); - try (ParallelTableIteratorOperation - dirTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, dirTable, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { - Function, Void> dirOperation = kv -> { - try { + Table dirTable = + omMetadataManager.getDirectoryTable(); + try (TableIterator> + dirTableIter = dirTable.iterator()) { + while (dirTableIter.hasNext()) { + Table.KeyValue kv = dirTableIter.next(); OmDirectoryInfo directoryInfo = kv.getValue(); - try { - lock.readLock().lock(); handlePutDirEvent(directoryInfo, nsSummaryMap); - } finally { - lock.readLock().unlock(); - } - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - try { - lock.writeLock().lock(); - if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { - throw new IOException("Unable to flush NSSummaryMap to DB"); - } - } finally { - lock.writeLock().unlock(); + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; } } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - return null; - }; - dirTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, dirOperation); + } } // Get fileTable used by FSO Table keyTable = omMetadataManager.getFileTable(); - try (ParallelTableIteratorOperation - keyTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { - Function, Void> keyOperation = kv -> { - try { + try (TableIterator> + keyTableIter = keyTable.iterator()) { + while (keyTableIter.hasNext()) { + Table.KeyValue kv = keyTableIter.next(); OmKeyInfo keyInfo = kv.getValue(); - try { - lock.readLock().lock(); handlePutKeyEvent(keyInfo, nsSummaryMap); - } finally { - lock.readLock().unlock(); - } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - try { - lock.writeLock().lock(); - if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { - throw new IOException("Unable to flush NSSummaryMap to DB"); + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; } - } finally { - lock.writeLock().unlock(); + } } } - } catch (IOException e) { - throw new RuntimeException(e); - } - return null; - }; - keyTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, keyOperation); - } - } catch (Exception ex) { + } catch (IOException ioEx) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", - ex); + ioEx); return false; } // flush and commit left out keys at end - if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { LOG.info("flushAndCommitNSToDB failed during reprocessWithFSO."); return false; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java index 91e8345df4bf..280fb2b8fddc 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java @@ -25,13 +25,9 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Function; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -44,7 +40,6 @@ import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; -import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,9 +55,6 @@ public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler { private final boolean enableFileSystemPaths; private final long nsSummaryFlushToDBMaxThreshold; - private final int maxKeysInMemory; - private final int maxIterators; - private final int maxWorkers; public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager reconNamespaceSummaryManager, @@ -70,10 +62,7 @@ public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager reconOMMetadataManager, OzoneConfiguration ozoneConfiguration, - long nsSummaryFlushToDBMaxThreshold, - int maxIterators, - int maxWorkers, - int maxKeysInMemory) { + long nsSummaryFlushToDBMaxThreshold) { super(reconNamespaceSummaryManager, reconOMMetadataManager); // true if FileSystemPaths enabled @@ -81,9 +70,6 @@ public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager .getBoolean(OmConfig.Keys.ENABLE_FILESYSTEM_PATHS, OmConfig.Defaults.ENABLE_FILESYSTEM_PATHS); this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; - this.maxIterators = maxIterators; - this.maxWorkers = maxWorkers; - this.maxKeysInMemory = maxKeysInMemory; } public Pair processWithLegacy(OMUpdateEventBatch events, @@ -146,7 +132,7 @@ public Pair processWithLegacy(OMUpdateEventBatch events, return new ImmutablePair<>(seekPos, false); } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { return new ImmutablePair<>(seekPos, false); } seekPos = eventCounter + 1; @@ -154,7 +140,7 @@ public Pair processWithLegacy(OMUpdateEventBatch events, } // flush and commit left out entries at end - if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { return new ImmutablePair<>(seekPos, false); } @@ -269,18 +255,17 @@ private void processWithObjectStoreLayout(OmKeyInfo updatedKeyInfo, } public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) { - Map nsSummaryMap = new ConcurrentHashMap<>(); + Map nsSummaryMap = new HashMap<>(); try { Table keyTable = omMetadataManager.getKeyTable(LEGACY_BUCKET_LAYOUT); - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - try (ParallelTableIteratorOperation - keyTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, StringCodec.get(), - maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { - Function, Void> keyOperation = kv -> { - try { + try (TableIterator> + keyTableIter = keyTable.iterator()) { + + while (keyTableIter.hasNext()) { + Table.KeyValue kv = keyTableIter.next(); OmKeyInfo keyInfo = kv.getValue(); // KeyTable entries belong to both Legacy and OBS buckets. @@ -288,11 +273,9 @@ public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) { // continue to the next iteration. if (!isBucketLayoutValid((ReconOMMetadataManager) omMetadataManager, keyInfo)) { - return null; + continue; } - try { - lock.readLock().lock(); if (enableFileSystemPaths) { // The LEGACY bucket is a file system bucket. setKeyParentID(keyInfo); @@ -312,36 +295,23 @@ public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) { // The LEGACY bucket is an object store bucket. setParentBucketId(keyInfo); handlePutKeyEvent(keyInfo, nsSummaryMap); - } - } finally { - lock.readLock().unlock(); } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - try { - lock.writeLock().lock(); - if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { - throw new IOException(); + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; } - } finally { - lock.writeLock().unlock(); + } } } - } catch (IOException e) { - throw new RuntimeException(e); - } - return null; - }; - keyTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, keyOperation); - } - } catch (Exception ex) { + } catch (IOException ioEx) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", - ex); + ioEx); nsSummaryMap.clear(); return false; } // flush and commit left out entries at end - if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { return false; } LOG.debug("Completed a reprocess run of NSSummaryTaskWithLegacy"); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java index b61880233d90..9a12c0e7b29f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java @@ -23,12 +23,8 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.function.Function; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -39,7 +35,6 @@ import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; -import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,35 +49,28 @@ public class NSSummaryTaskWithOBS extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithOBS.class); private final long nsSummaryFlushToDBMaxThreshold; - private final int maxKeysInMemory; - private final int maxIterators; - private final int maxWorkers; public NSSummaryTaskWithOBS( ReconNamespaceSummaryManager reconNamespaceSummaryManager, ReconOMMetadataManager reconOMMetadataManager, - long nsSummaryFlushToDBMaxThreshold, int maxIterators, int maxWorkers, int maxKeysInMemory) { + long nsSummaryFlushToDBMaxThreshold) { super(reconNamespaceSummaryManager, reconOMMetadataManager); this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; - this.maxIterators = maxIterators; - this.maxWorkers = maxWorkers; - this.maxKeysInMemory = maxKeysInMemory; } public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { - Map nsSummaryMap = new ConcurrentHashMap<>(); - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + Map nsSummaryMap = new HashMap<>(); + try { Table keyTable = omMetadataManager.getKeyTable(BUCKET_LAYOUT); - try (ParallelTableIteratorOperation - keyTableIter = new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, StringCodec.get(), - maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + try (TableIterator> + keyTableIter = keyTable.iterator()) { - Function, Void> keyOperation = kv -> { - try { + while (keyTableIter.hasNext()) { + Table.KeyValue kv = keyTableIter.next(); OmKeyInfo keyInfo = kv.getValue(); // KeyTable entries belong to both Legacy and OBS buckets. @@ -97,44 +85,28 @@ public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { .getBucketTable().getSkipCache(bucketDBKey); if (omBucketInfo.getBucketLayout() != BUCKET_LAYOUT) { - return null; + continue; } setKeyParentID(keyInfo); - try { - lock.readLock().lock(); handlePutKeyEvent(keyInfo, nsSummaryMap); - } catch (IOException e) { - throw new RuntimeException(e); - } finally { - lock.readLock().unlock(); - } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - try { - lock.writeLock().lock(); - if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { - throw new IOException("Failed to commit nsSummaryMap"); + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; } - } finally { - lock.writeLock().unlock(); + } } } - } catch (IOException e) { - throw new RuntimeException(e); - } - return null; - }; - keyTableIter.performTaskOnTableVals(this.getClass().getName(), null, null, keyOperation); - } - } catch (Exception exception) { - LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", exception); + } catch (IOException ioEx) { + LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", + ioEx); nsSummaryMap.clear(); return false; } // flush and commit left out entries at end - if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { return false; } LOG.debug("Completed a reprocess run of NSSummaryTaskWithOBS"); @@ -224,7 +196,7 @@ public Pair processWithOBS(OMUpdateEventBatch events, LOG.debug("Skipping DB update event: {}", action); } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap, nsSummaryFlushToDBMaxThreshold)) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { return new ImmutablePair<>(seekPos, false); } seekPos = eventCounter + 1; @@ -238,7 +210,7 @@ public Pair processWithOBS(OMUpdateEventBatch events, } // Flush and commit left-out entries at the end - if (!flushAndCommitNSToDB(nsSummaryMap, 0)) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { return new ImmutablePair<>(seekPos, false); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index b41c4feaae1d..4794ecf1f309 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -481,7 +481,7 @@ public void testGetKeysForContainer() throws IOException { setUpFSOData(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); // Reprocess the container key mapper to ensure the latest mapping is used reprocessContainerKeyMapper(); @@ -569,7 +569,7 @@ public void testGetKeysForContainerWithPrevKey() throws IOException { reprocessContainerKeyMapper(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); response = containerEndpoint.getKeysForContainer(20L, -1, "/0/1/2/file7"); @@ -1826,7 +1826,7 @@ public void testDuplicateFSOKeysForContainerEndpoint() throws IOException { setUpDuplicateFSOFileKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); // Reprocess the container key mappings so that the new keys are loaded. reprocessContainerKeyMapper(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java index c9f6c5dbf182..2d375b6fe866 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryDiskUsageOrdering.java @@ -105,7 +105,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java index 90494ef02c4f..557115150d28 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java @@ -434,7 +434,7 @@ public void setUp() throws Exception { setUpMultiBlockReplicatedKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java index 7546e649afad..8844b96c0226 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java @@ -435,7 +435,7 @@ public void setUp() throws Exception { setUpMultiBlockReplicatedKeys(); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, - reconOMMetadataManager, conf, 10, 1, 1, 10); + reconOMMetadataManager, conf, 10); nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java index 8392a6c51187..eefc11940605 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java @@ -371,11 +371,11 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, - reconOMMetadataManager, conf, 10, 1, 1, 10); + reconOMMetadataManager, conf, 10); nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java index 0729f8c18b7c..c83501bb7a0d 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java @@ -328,13 +328,13 @@ public void setUp() throws Exception { setUpOmData(); NSSummaryTaskWithLegacy nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration, 10, 1, 1, 10); + reconOMMetadataManager, ozoneConfiguration, 10); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); NSSummaryTaskWithFSO nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); reconNamespaceSummaryManager.clearNSSummaryTable(); nSSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java index c53b1876c743..b59aaa3c8b8f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java @@ -113,7 +113,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10, 1, 1, 10); + reconOMMetadataManager, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java index 292dc76e9775..e93f9176c142 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java @@ -81,7 +81,7 @@ void setUp(@TempDir File tmpDir) throws Exception { nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - threshold, 1, 1, 10); + threshold); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java index 3ef325a8bb57..ecd886f276b6 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java @@ -65,7 +65,7 @@ void setUp(@TempDir File tmpDir) throws Exception { long threshold = getOmConfiguration().getLong(OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, 10); nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - getOmConfiguration(), threshold, 1, 1, 10); + getOmConfiguration(), threshold); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java index cc0254622c4f..795e5d1da334 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java @@ -61,7 +61,7 @@ void setUp(@TempDir File tmpDir) throws Exception { nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(getReconNamespaceSummaryManager(), getReconOMMetadataManager(), getOzoneConfiguration(), - threshold, 1, 1, 10); + threshold); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java index bf0c0c22775d..8d44633ba7ff 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithOBS.java @@ -61,7 +61,7 @@ false, getBucketLayout(), OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); nSSummaryTaskWithOBS = new NSSummaryTaskWithOBS(getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - threshold, 1, 1, 10); + threshold); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java index 14cb28f7761a..3dd691110611 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTreePrecomputeValues.java @@ -112,7 +112,7 @@ void setUp(@TempDir File tmpDir) throws Exception { nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( getReconNamespaceSummaryManager(), getReconOMMetadataManager(), - threshold, 1, 1, 10); + threshold); // Populate a custom complex tree structure populateComplexTree(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java index 27c5242087c6..659ece4b3987 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java @@ -144,7 +144,7 @@ private void initializeInjector() throws IOException { omTableInsightTask = new OmTableInsightTask( reconGlobalStatsManager, reconOMMetadataManager); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( - reconNamespaceSummaryManager, reconOMMetadataManager, 10, 1, 1, 10); + reconNamespaceSummaryManager, reconOMMetadataManager, 10); dslContext = getDslContext(); omTableInsightTask.setTables(omTableInsightTask.getTaskTables()); From 1b39a9a05f839fb830401ec9931610d5f69855dd Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 5 Nov 2025 14:04:41 +0530 Subject: [PATCH 03/49] Removed unnecessary changes --- .../ozone/recon/tasks/NSSummaryTask.java | 6 +- .../tasks/NSSummaryTaskDbEventHandler.java | 26 +++---- .../recon/tasks/NSSummaryTaskWithFSO.java | 70 +++++++++---------- .../recon/tasks/NSSummaryTaskWithLegacy.java | 60 ++++++++-------- .../recon/tasks/NSSummaryTaskWithOBS.java | 42 +++++------ 5 files changed, 101 insertions(+), 103 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index ebd0f6f67e12..6b4a5cce0e71 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java @@ -43,7 +43,6 @@ import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; -import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -217,7 +216,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { return buildTaskResult(false); } - LOG.info("Starting RocksDB Reprocess for {}", getTaskName()); + LOG.info("Starting NSSummary tree reprocess with unified control..."); long startTime = System.nanoTime(); // Record start time try { @@ -235,7 +234,6 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { protected TaskResult executeReprocess(OMMetadataManager omMetadataManager, long startTime) { // Initialize a list of tasks to run in parallel Collection> tasks = new ArrayList<>(); - LOG.info("Starting RocksDB Reprocess for {}", getTaskName()); try { // reinit Recon RocksDB's namespace CF. @@ -307,7 +305,7 @@ protected TaskResult executeReprocess(OMMetadataManager omMetadataManager, long // Reset state to IDLE on successful completion if (success) { REBUILD_STATE.set(RebuildState.IDLE); - LOG.info("Completed RocksDB Reprocess for {} in {}", getTaskName(), (endTime - startTime)); + LOG.info("NSSummary tree reprocess completed successfully with unified control."); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java index c28054ccd169..e83fc64ad4cc 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java @@ -97,11 +97,11 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) { - try { + try { updateNSSummariesToDB(nsSummaryMap, Collections.emptyList()); - } catch (IOException e) { - LOG.error("Unable to write Namespace Summary data in Recon DB.", e); - return false; - } finally { - nsSummaryMap.clear(); - } + } catch (IOException e) { + LOG.error("Unable to write Namespace Summary data in Recon DB.", e); + return false; + } finally { + nsSummaryMap.clear(); + } return true; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java index b0d38419ba53..0e7f41c16b56 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java @@ -82,7 +82,7 @@ public Pair processWithFSO(OMUpdateEventBatch events, final Collection objectIdsToBeDeleted = Collections.synchronizedList(new ArrayList<>()); while (eventIterator.hasNext()) { OMDBUpdateEvent omdbUpdateEvent = eventIterator.next(); + WithParentObjectId> omdbUpdateEvent = eventIterator.next(); OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction(); eventCounter++; @@ -134,35 +134,35 @@ public Pair processWithFSO(OMUpdateEventBatch events, private void handleUpdateOnDirTable(OMDBUpdateEvent omdbUpdateEvent, OMDBUpdateEvent.OMDBUpdateAction action, Map nsSummaryMap) throws IOException { - OMDBUpdateEvent dirTableUpdateEvent = - (OMDBUpdateEvent) omdbUpdateEvent; - OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue(); - OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue(); + OMDBUpdateEvent dirTableUpdateEvent = + (OMDBUpdateEvent) omdbUpdateEvent; + OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue(); + OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue(); - switch (action) { - case PUT: - handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); - break; + switch (action) { + case PUT: + handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); + break; - case DELETE: - handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap); - break; + case DELETE: + handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap); + break; - case UPDATE: - if (oldDirectoryInfo != null) { - // delete first, then put - handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap); - } else { + case UPDATE: + if (oldDirectoryInfo != null) { + // delete first, then put + handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap); + } else { LOG.warn("Update event does not have the old dirInfo for {}.", dirTableUpdateEvent.getKey()); - } - handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); - break; + } + handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap); + break; - default: - LOG.debug("Skipping DB update event : {}", - omdbUpdateEvent.getAction()); - } - } + default: + LOG.debug("Skipping DB update event : {}", + omdbUpdateEvent.getAction()); + } + } private void handleUpdateOnDeletedDirTable(OMDBUpdateEvent omdbUpdateEvent, OMDBUpdateEvent.OMDBUpdateAction action, Map nsSummaryMap, @@ -232,13 +232,13 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { dirTableIter = dirTable.iterator()) { while (dirTableIter.hasNext()) { Table.KeyValue kv = dirTableIter.next(); - OmDirectoryInfo directoryInfo = kv.getValue(); - handlePutDirEvent(directoryInfo, nsSummaryMap); - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + OmDirectoryInfo directoryInfo = kv.getValue(); + handlePutDirEvent(directoryInfo, nsSummaryMap); + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { if (!flushAndCommitNSToDB(nsSummaryMap)) { return false; - } } + } } } @@ -250,15 +250,15 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { keyTableIter = keyTable.iterator()) { while (keyTableIter.hasNext()) { Table.KeyValue kv = keyTableIter.next(); - OmKeyInfo keyInfo = kv.getValue(); - handlePutKeyEvent(keyInfo, nsSummaryMap); - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + OmKeyInfo keyInfo = kv.getValue(); + handlePutKeyEvent(keyInfo, nsSummaryMap); + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { if (!flushAndCommitNSToDB(nsSummaryMap)) { return false; - } - } - } } + } + } + } } catch (IOException ioEx) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java index 280fb2b8fddc..6ad94797603b 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java @@ -266,43 +266,43 @@ public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) { while (keyTableIter.hasNext()) { Table.KeyValue kv = keyTableIter.next(); - OmKeyInfo keyInfo = kv.getValue(); + OmKeyInfo keyInfo = kv.getValue(); - // KeyTable entries belong to both Legacy and OBS buckets. - // Check bucket layout and if it's OBS - // continue to the next iteration. - if (!isBucketLayoutValid((ReconOMMetadataManager) omMetadataManager, - keyInfo)) { + // KeyTable entries belong to both Legacy and OBS buckets. + // Check bucket layout and if it's OBS + // continue to the next iteration. + if (!isBucketLayoutValid((ReconOMMetadataManager) omMetadataManager, + keyInfo)) { continue; - } + } - if (enableFileSystemPaths) { - // The LEGACY bucket is a file system bucket. - setKeyParentID(keyInfo); - - if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) { - OmDirectoryInfo directoryInfo = - new OmDirectoryInfo.Builder() - .setName(keyInfo.getKeyName()) - .setObjectID(keyInfo.getObjectID()) - .setParentObjectID(keyInfo.getParentObjectID()) - .build(); - handlePutDirEvent(directoryInfo, nsSummaryMap); - } else { - handlePutKeyEvent(keyInfo, nsSummaryMap); - } - } else { - // The LEGACY bucket is an object store bucket. - setParentBucketId(keyInfo); - handlePutKeyEvent(keyInfo, nsSummaryMap); + if (enableFileSystemPaths) { + // The LEGACY bucket is a file system bucket. + setKeyParentID(keyInfo); + + if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) { + OmDirectoryInfo directoryInfo = + new OmDirectoryInfo.Builder() + .setName(keyInfo.getKeyName()) + .setObjectID(keyInfo.getObjectID()) + .setParentObjectID(keyInfo.getParentObjectID()) + .build(); + handlePutDirEvent(directoryInfo, nsSummaryMap); + } else { + handlePutKeyEvent(keyInfo, nsSummaryMap); } - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + } else { + // The LEGACY bucket is an object store bucket. + setParentBucketId(keyInfo); + handlePutKeyEvent(keyInfo, nsSummaryMap); + } + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { if (!flushAndCommitNSToDB(nsSummaryMap)) { return false; - } - } - } } + } + } + } } catch (IOException ioEx) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", ioEx); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java index 9a12c0e7b29f..e1c6f4542c46 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java @@ -71,33 +71,33 @@ public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { while (keyTableIter.hasNext()) { Table.KeyValue kv = keyTableIter.next(); - OmKeyInfo keyInfo = kv.getValue(); - - // KeyTable entries belong to both Legacy and OBS buckets. - // Check bucket layout and if it's anything other than OBS, - // continue to the next iteration. - String volumeName = keyInfo.getVolumeName(); - String bucketName = keyInfo.getBucketName(); - String bucketDBKey = omMetadataManager - .getBucketKey(volumeName, bucketName); - // Get bucket info from bucket table - OmBucketInfo omBucketInfo = omMetadataManager - .getBucketTable().getSkipCache(bucketDBKey); - - if (omBucketInfo.getBucketLayout() != BUCKET_LAYOUT) { + OmKeyInfo keyInfo = kv.getValue(); + + // KeyTable entries belong to both Legacy and OBS buckets. + // Check bucket layout and if it's anything other than OBS, + // continue to the next iteration. + String volumeName = keyInfo.getVolumeName(); + String bucketName = keyInfo.getBucketName(); + String bucketDBKey = omMetadataManager + .getBucketKey(volumeName, bucketName); + // Get bucket info from bucket table + OmBucketInfo omBucketInfo = omMetadataManager + .getBucketTable().getSkipCache(bucketDBKey); + + if (omBucketInfo.getBucketLayout() != BUCKET_LAYOUT) { continue; - } + } - setKeyParentID(keyInfo); + setKeyParentID(keyInfo); - handlePutKeyEvent(keyInfo, nsSummaryMap); - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + handlePutKeyEvent(keyInfo, nsSummaryMap); + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { if (!flushAndCommitNSToDB(nsSummaryMap)) { return false; - } - } - } } + } + } + } } catch (IOException ioEx) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", ioEx); From 266138a7933bc7b0ca3f7afe754432ed2a428ab7 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 7 Nov 2025 11:01:19 +0530 Subject: [PATCH 04/49] Refactored and changed the code a bit --- .../impl/OzoneManagerServiceProviderImpl.java | 280 +++++++++--------- .../recon/tasks/ContainerKeyMapperHelper.java | 12 +- .../recon/tasks/FileSizeCountTaskFSO.java | 15 +- .../recon/tasks/FileSizeCountTaskHelper.java | 93 +++--- .../recon/tasks/FileSizeCountTaskOBS.java | 15 +- .../ozone/recon/tasks/OmTableInsightTask.java | 103 ++++++- .../util/ParallelTableIteratorOperation.java | 3 + 7 files changed, 332 insertions(+), 189 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 3cd785e53a59..9751a63eae31 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -114,7 +114,7 @@ @Singleton public class OzoneManagerServiceProviderImpl implements OzoneManagerServiceProvider { - + private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; @@ -702,145 +702,153 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = false; + boolean fullSnapshot = true; + - if (currentSequenceNumber <= 0) { - fullSnapshot = true; + if (reInitializeTasksCalled.compareAndSet(false, true)) { + LOG.info("Calling reprocess on Recon tasks."); + reconTaskController.reInitializeTasks(omMetadataManager, null); } else { - // Get updates from OM and apply to local Recon OM DB and update task status in table - deltaReconTaskStatusUpdater.recordRunStart(); - int loopCount = 0; - long fromSequenceNumber = currentSequenceNumber; - long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; - /** - * This loop will continue to fetch and apply OM DB updates and with every - * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. - * It continues to fetch from OM till the lag, between OM DB WAL sequence number - * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. - * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. - */ - while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { - try (OMDBUpdatesHandler omdbUpdatesHandler = - new OMDBUpdatesHandler(omMetadataManager)) { - - // If interrupt was previously signalled, - // we should check for it before starting delta update sync. - if (Thread.currentThread().isInterrupted()) { - throw new InterruptedException("Thread interrupted during delta update."); - } - diffBetweenOMDbAndReconDBSeqNumber = - getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); - deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); - // Keeping last updated sequence number for both full and delta tasks to be same - // because sequence number of DB denotes and points to same OM DB copy of Recon, - // even though two different tasks are updating the DB at different conditions, but - // it tells the sync state with actual OM DB for the same Recon OM DB copy. - deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - deltaReconTaskStatusUpdater.recordRunCompletion(); - fullSnapshotReconTaskUpdater.updateDetails(); - // Update the current OM metadata manager in task controller - reconTaskController.updateOMMetadataManager(omMetadataManager); - - // Pass on DB update events to tasks that are listening. - reconTaskController.consumeOMEvents(new OMUpdateEventBatch( - omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); - - // Check if task reinitialization is needed due to buffer overflow or task failures - boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); - boolean tasksFailed = reconTaskController.hasTasksFailed(); - - if (bufferOverflowed || tasksFailed) { - ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? - ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : - ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; - - LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", - reason); - - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - - // Queue async reinitialization event - checkpoint creation and retry logic is handled internally - ReconTaskController.ReInitializationResult result = - reconTaskController.queueReInitializationEvent(reason); - - //TODO: Create a metric to track this event buffer overflow or task failure event - boolean triggerFullSnapshot = - Optional.ofNullable(result) - .map(r -> { - switch (r) { - case MAX_RETRIES_EXCEEDED: - LOG.warn( - "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + - "fallback"); - return true; - - case RETRY_LATER: - LOG.debug("Reinitialization event queueing will be retried in next iteration"); - return false; - - default: - LOG.info("Reinitialization event successfully queued"); - return false; - } - }) - .orElseGet(() -> { - LOG.error( - "ReInitializationResult is null, something went wrong in queueing reinitialization " + - "event"); - return true; - }); - - if (triggerFullSnapshot) { - fullSnapshot = true; - } - } - currentSequenceNumber = getCurrentOMDBSequenceNumber(); - LOG.debug("Updated current sequence number: {}", currentSequenceNumber); - loopCount++; - } catch (InterruptedException intEx) { - LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); - // We are updating the table even if it didn't run i.e. got interrupted beforehand - // to indicate that a task was supposed to run, but it didn't. - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - Thread.currentThread().interrupt(); - // Since thread is interrupted, we do not fall back to snapshot sync. - // Return with sync failed status. - return false; - } catch (Exception e) { - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", - e.getMessage()); - fullSnapshot = true; - } - if (fullSnapshot) { - break; - } - } - LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, - getCurrentOMDBSequenceNumber() - fromSequenceNumber); + LOG.info("reInitializeTasks already called once; skipping."); } - if (fullSnapshot) { - try { - executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); - } catch (InterruptedException intEx) { - LOG.error("OM DB Snapshot update sync thread was interrupted."); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - Thread.currentThread().interrupt(); - // Mark sync status as failed. - return false; - } catch (Exception e) { - metrics.incrNumSnapshotRequestsFailed(); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - LOG.error("Unable to update Recon's metadata with new OM DB. ", e); - // Update health status in ReconContext - reconContext.updateHealthStatus(new AtomicBoolean(false)); - reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); - } - } +// if (currentSequenceNumber <= 0) { +// fullSnapshot = true; +// } else { +// // Get updates from OM and apply to local Recon OM DB and update task status in table +// deltaReconTaskStatusUpdater.recordRunStart(); +// int loopCount = 0; +// long fromSequenceNumber = currentSequenceNumber; +// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; +// /** +// * This loop will continue to fetch and apply OM DB updates and with every +// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. +// * It continues to fetch from OM till the lag, between OM DB WAL sequence number +// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. +// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. +// */ +// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { +// try (OMDBUpdatesHandler omdbUpdatesHandler = +// new OMDBUpdatesHandler(omMetadataManager)) { +// +// // If interrupt was previously signalled, +// // we should check for it before starting delta update sync. +// if (Thread.currentThread().isInterrupted()) { +// throw new InterruptedException("Thread interrupted during delta update."); +// } +// diffBetweenOMDbAndReconDBSeqNumber = +// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); +// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); +// // Keeping last updated sequence number for both full and delta tasks to be same +// // because sequence number of DB denotes and points to same OM DB copy of Recon, +// // even though two different tasks are updating the DB at different conditions, but +// // it tells the sync state with actual OM DB for the same Recon OM DB copy. +// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// deltaReconTaskStatusUpdater.recordRunCompletion(); +// fullSnapshotReconTaskUpdater.updateDetails(); +// // Update the current OM metadata manager in task controller +// reconTaskController.updateOMMetadataManager(omMetadataManager); +// +// // Pass on DB update events to tasks that are listening. +// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( +// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); +// +// // Check if task reinitialization is needed due to buffer overflow or task failures +// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); +// boolean tasksFailed = reconTaskController.hasTasksFailed(); +// +// if (bufferOverflowed || tasksFailed) { +// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? +// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : +// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; +// +// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", +// reason); +// +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// +// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally +// ReconTaskController.ReInitializationResult result = +// reconTaskController.queueReInitializationEvent(reason); +// +// //TODO: Create a metric to track this event buffer overflow or task failure event +// boolean triggerFullSnapshot = +// Optional.ofNullable(result) +// .map(r -> { +// switch (r) { +// case MAX_RETRIES_EXCEEDED: +// LOG.warn( +// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + +// "fallback"); +// return true; +// +// case RETRY_LATER: +// LOG.debug("Reinitialization event queueing will be retried in next iteration"); +// return false; +// +// default: +// LOG.info("Reinitialization event successfully queued"); +// return false; +// } +// }) +// .orElseGet(() -> { +// LOG.error( +// "ReInitializationResult is null, something went wrong in queueing reinitialization " + +// "event"); +// return true; +// }); +// +// if (triggerFullSnapshot) { +// fullSnapshot = true; +// } +// } +// currentSequenceNumber = getCurrentOMDBSequenceNumber(); +// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); +// loopCount++; +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); +// // We are updating the table even if it didn't run i.e. got interrupted beforehand +// // to indicate that a task was supposed to run, but it didn't. +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// Thread.currentThread().interrupt(); +// // Since thread is interrupted, we do not fall back to snapshot sync. +// // Return with sync failed status. +// return false; +// } catch (Exception e) { +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", +// e.getMessage()); +// fullSnapshot = true; +// } +// if (fullSnapshot) { +// break; +// } +// } +// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, +// getCurrentOMDBSequenceNumber() - fromSequenceNumber); +// } + +// if (fullSnapshot) { +// try { +// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Snapshot update sync thread was interrupted."); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// Thread.currentThread().interrupt(); +// // Mark sync status as failed. +// return false; +// } catch (Exception e) { +// metrics.incrNumSnapshotRequestsFailed(); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); +// // Update health status in ReconContext +// reconContext.updateHealthStatus(new AtomicBoolean(false)); +// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); +// } +// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 14ab2ff9cc71..413fa2fa4471 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -100,7 +100,8 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, Map containerKeyCountMap = new ConcurrentHashMap<>(); try { - LOG.info("Starting RocksDB Reprocess for {}", taskName); + LOG.info("{}: Starting parallel reprocess with {} iterators, {} workers, max {} keys in memory for bucket layout {}", + taskName, maxIterators, maxWorkers, maxKeysInMemory, bucketLayout); Instant start = Instant.now(); // Ensure the tables are truncated only once @@ -152,8 +153,13 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, Instant end = Instant.now(); long durationMillis = Duration.between(start, end).toMillis(); double durationSeconds = (double) durationMillis / 1000.0; - LOG.info("Completed RocksDB Reprocess for {}. Processed {} keys in {} ms ({} seconds).", - taskName, omKeyCount, durationMillis, durationSeconds); + long keysProcessed = omKeyCount.get(); + double throughput = keysProcessed / Math.max(durationSeconds, 0.001); + + LOG.info("{}: Parallel reprocess completed. Processed {} keys in {} ms ({} sec) - " + + "Throughput: {} keys/sec - Containers: {}, Container-Key mappings: {}", + taskName, keysProcessed, durationMillis, String.format("%.2f", durationSeconds), + String.format("%.2f", throughput), containerKeyCountMap.size(), containerKeyMap.size()); } catch (Exception ex) { LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ex); return false; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java index 7a13fea4f989..5f3118b3d675 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconFileMetadataManager; @@ -53,11 +54,23 @@ public ReconOmTask getStagedTask(ReconOMMetadataManager stagedOmMetadataManager, @Override public TaskResult reprocess(OMMetadataManager omMetadataManager) { + int maxKeysInMemory = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT); + int maxIterators = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); + int maxWorkers = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); return FileSizeCountTaskHelper.reprocess( omMetadataManager, reconFileMetadataManager, BucketLayout.FILE_SYSTEM_OPTIMIZED, - getTaskName() + getTaskName(), + maxIterators, + maxWorkers, + maxKeysInMemory ); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index 4eef7ee6916c..78bdd7005467 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -18,10 +18,14 @@ package org.apache.hadoop.ozone.recon.tasks; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; +import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -30,6 +34,7 @@ import org.apache.hadoop.ozone.recon.ReconConstants; import org.apache.hadoop.ozone.recon.ReconUtils; import org.apache.hadoop.ozone.recon.spi.ReconFileMetadataManager; +import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,9 +53,8 @@ public abstract class FileSizeCountTaskHelper { */ public static void handlePutKeyEvent(OmKeyInfo omKeyInfo, Map fileSizeCountMap) { - FileSizeCountKey key = getFileSizeCountKey(omKeyInfo); - Long count = fileSizeCountMap.containsKey(key) ? fileSizeCountMap.get(key) + 1L : 1L; - fileSizeCountMap.put(key, count); + fileSizeCountMap.compute(getFileSizeCountKey(omKeyInfo), + (k, v) -> (v == null ? 0L : v) + 1L); } /** @@ -61,9 +65,8 @@ public static void handleDeleteKeyEvent(String key, OmKeyInfo omKeyInfo, if (omKeyInfo == null) { LOG.warn("Deleting a key not found while handling DELETE key event. Key not found in Recon OM DB: {}", key); } else { - FileSizeCountKey countKey = getFileSizeCountKey(omKeyInfo); - Long count = fileSizeCountMap.containsKey(countKey) ? fileSizeCountMap.get(countKey) - 1L : -1L; - fileSizeCountMap.put(countKey, count); + fileSizeCountMap.compute(getFileSizeCountKey(omKeyInfo), + (k, v) -> (v == null ? 0L : v) - 1L); } } @@ -105,24 +108,39 @@ public static void truncateFileCountTableIfNeeded(ReconFileMetadataManager recon public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManager, ReconFileMetadataManager reconFileMetadataManager, BucketLayout bucketLayout, - String taskName) { - LOG.info("Starting RocksDB Reprocess for {}", taskName); - Map fileSizeCountMap = new HashMap<>(); - long startTime = Time.monotonicNow(); + String taskName, + int maxIterators, + int maxWorkers, + int maxKeysInMemory) { + LOG.info("{}: Starting parallel RocksDB reprocess with {} iterators, {} workers for bucket layout {}", + taskName, maxIterators, maxWorkers, bucketLayout); + Map fileSizeCountMap = new ConcurrentHashMap<>(); + long overallStartTime = Time.monotonicNow(); // Ensure the file count table is truncated only once during reprocess truncateFileCountTableIfNeeded(reconFileMetadataManager, taskName); + long iterationStartTime = Time.monotonicNow(); boolean status = reprocessBucketLayout( - bucketLayout, omMetadataManager, fileSizeCountMap, reconFileMetadataManager, taskName); + bucketLayout, omMetadataManager, fileSizeCountMap, reconFileMetadataManager, taskName, + maxIterators, maxWorkers, maxKeysInMemory); if (!status) { return buildTaskResult(taskName, false); } + long iterationEndTime = Time.monotonicNow(); + long writeStartTime = Time.monotonicNow(); writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + long writeEndTime = Time.monotonicNow(); - long endTime = Time.monotonicNow(); - LOG.info("Completed RocksDB Reprocess for {} in {}", taskName, (endTime - startTime)); + long overallEndTime = Time.monotonicNow(); + long totalDurationMs = overallEndTime - overallStartTime; + long iterationDurationMs = iterationEndTime - iterationStartTime; + long writeDurationMs = writeEndTime - writeStartTime; + + LOG.info("{}: Parallel RocksDB reprocess completed - Total: {} ms (Iteration: {} ms, Write: {} ms) - " + + "File count entries: {}", taskName, totalDurationMs, iterationDurationMs, writeDurationMs, + fileSizeCountMap.size()); return buildTaskResult(taskName, true); } @@ -134,31 +152,40 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, OMMetadataManager omMetadataManager, Map 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 omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); - int totalKeysProcessed = 0; + long startTime = Time.monotonicNow(); - try (TableIterator> keyIter = - omKeyInfoTable.iterator()) { - while (keyIter.hasNext()) { - Table.KeyValue kv = keyIter.next(); - 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(); - } - } - } catch (IOException ioEx) { - LOG.error("Unable to populate File Size Count for {} in RocksDB.", taskName, ioEx); + // Use parallel table iteration + Function, Void> kvOperation = kv -> { + handlePutKeyEvent(kv.getValue(), fileSizeCountMap); + return null; + }; + + try (ParallelTableIteratorOperation keyIter = + new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, 100000)) { + keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); + } catch (Exception ex) { + LOG.error("Unable to populate File Size Count for {} in RocksDB.", taskName, ex); return false; } - LOG.info("Reprocessed {} keys for bucket layout {} using RocksDB.", totalKeysProcessed, bucketLayout); + long endTime = Time.monotonicNow(); + long durationMs = endTime - startTime; + double durationSec = durationMs / 1000.0; + int totalKeys = fileSizeCountMap.values().stream().mapToInt(Long::intValue).sum(); + double throughput = totalKeys / Math.max(durationSec, 0.001); + + LOG.info("{}: Reprocessed {} keys for bucket layout {} in {} ms ({} sec) - Throughput: {}/sec", + taskName, totalKeys, bucketLayout, durationMs, String.format("%.2f", durationSec), + String.format("%.2f", throughput)); + return true; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java index fc192eca0422..c4cf29b22e43 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.helpers.BucketLayout; +import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconFileMetadataManager; @@ -53,11 +54,23 @@ public ReconOmTask getStagedTask(ReconOMMetadataManager stagedOmMetadataManager, @Override public TaskResult reprocess(OMMetadataManager omMetadataManager) { + int maxKeysInMemory = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT); + int maxIterators = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); + int maxWorkers = ozoneConfiguration.getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); return FileSizeCountTaskHelper.reprocess( omMetadataManager, reconFileMetadataManager, BucketLayout.OBJECT_STORE, - getTaskName() + getTaskName(), + maxIterators, + maxWorkers, + maxKeysInMemory ); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index cf81b4e35b59..d4e9b151f0fd 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java @@ -31,14 +31,20 @@ import java.util.Iterator; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; +import org.apache.hadoop.hdds.utils.db.StringCodec; 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.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; +import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconGlobalStatsManager; +import org.apache.hadoop.ozone.recon.tasks.util.ParallelTableIteratorOperation; import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,6 +64,9 @@ public class OmTableInsightTask implements ReconOmTask { private Map objectCountMap; private Map unReplicatedSizeMap; private Map replicatedSizeMap; + private final int maxKeysInMemory; + private final int maxIterators; + private final int maxWorkers; @Inject public OmTableInsightTask(ReconGlobalStatsManager reconGlobalStatsManager, @@ -70,6 +79,15 @@ public OmTableInsightTask(ReconGlobalStatsManager reconGlobalStatsManager, tableHandlers.put(OPEN_KEY_TABLE, new OpenKeysInsightHandler()); tableHandlers.put(OPEN_FILE_TABLE, new OpenKeysInsightHandler()); tableHandlers.put(DELETED_TABLE, new DeletedKeysInsightHandler()); + this.maxKeysInMemory = reconOMMetadataManager.getOzoneConfiguration().getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_KEYS_IN_MEMORY_DEFAULT); + this.maxIterators = reconOMMetadataManager.getOzoneConfiguration().getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); + this.maxWorkers = reconOMMetadataManager.getOzoneConfiguration().getInt( + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, + ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); } @Override @@ -115,22 +133,26 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { for (String tableName : tables) { Table table = omMetadataManager.getTable(tableName); - try (TableIterator> iterator - = table.iterator()) { + try { if (tableHandlers.containsKey(tableName)) { - Triple details = - tableHandlers.get(tableName).getTableSizeAndCount(iterator); - objectCountMap.put(getTableCountKeyFromTable(tableName), - details.getLeft()); - unReplicatedSizeMap.put( - getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); - replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), - details.getRight()); + // Parallelize handler tables (OPEN_KEY_TABLE, OPEN_FILE_TABLE, DELETED_TABLE) + LOG.info("{}: Processing table {} with parallel iteration ({} iterators, {} workers)", + getTaskName(), tableName, maxIterators, maxWorkers); + + Triple details = getTableSizeAndCountParallel( + omMetadataManager, tableName, tableHandlers.get(tableName)); + + objectCountMap.put(getTableCountKeyFromTable(tableName), details.getLeft()); + unReplicatedSizeMap.put(getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); + replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), details.getRight()); } else { - long count = Iterators.size(iterator); - objectCountMap.put(getTableCountKeyFromTable(tableName), count); + // Keep simple tables sequential due to heterogeneous key types + try (TableIterator> iterator = table.iterator()) { + long count = Iterators.size(iterator); + objectCountMap.put(getTableCountKeyFromTable(tableName), count); + } } - } catch (IOException ioEx) { + } catch (Exception ioEx) { LOG.error("Unable to populate Table Count in Recon DB.", ioEx); return buildTaskResult(false); } @@ -146,11 +168,62 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { writeDataToDB(replicatedSizeMap); } long endTime = Time.monotonicNow(); - - LOG.info("Completed RocksDB Reprocess for {} in {}", getTaskName(), (endTime - startTime)); + long durationMs = endTime - startTime; + double durationSec = durationMs / 1000.0; + int parallelizedTables = tableHandlers.size(); // 3 handler tables + int sequentialTables = tables.size() - parallelizedTables; + + LOG.info("{}: RocksDB reprocess completed in {} ms ({} sec) - " + + "Total tables: {}, Parallelized (with handlers): {}, Sequential (simple count): {}", + getTaskName(), durationMs, String.format("%.2f", durationSec), + tables.size(), parallelizedTables, sequentialTables); return buildTaskResult(true); } + /** + * Parallel version of getTableSizeAndCount for handler tables. + * Uses ParallelTableIteratorOperation with atomic accumulators. + */ + private Triple getTableSizeAndCountParallel( + OMMetadataManager omMetadataManager, String tableName, OmTableHandler handler) + throws Exception { + + AtomicLong count = new AtomicLong(0); + AtomicLong unReplicatedSize = new AtomicLong(0); + AtomicLong replicatedSize = new AtomicLong(0); + + Table table = omMetadataManager.getTable(tableName); + + try (ParallelTableIteratorOperation parallelIter = + new ParallelTableIteratorOperation<>(omMetadataManager, table, + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, 100000)) { + + parallelIter.performTaskOnTableVals(tableName, null, null, kv -> { + if (kv != null && kv.getValue() != null) { + Object value = kv.getValue(); + + if (value instanceof OmKeyInfo) { + // For OPEN_KEY_TABLE and OPEN_FILE_TABLE + OmKeyInfo omKeyInfo = (OmKeyInfo) value; + unReplicatedSize.addAndGet(omKeyInfo.getDataSize()); + replicatedSize.addAndGet(omKeyInfo.getReplicatedSize()); + count.incrementAndGet(); + } else if (value instanceof RepeatedOmKeyInfo) { + // For DELETED_TABLE - match original getTableSizeAndCount() logic (lines 129-130) + RepeatedOmKeyInfo repeatedOmKeyInfo = (RepeatedOmKeyInfo) value; + org.apache.commons.lang3.tuple.Pair sizes = repeatedOmKeyInfo.getTotalSize(); + unReplicatedSize.addAndGet(sizes.getRight()); // Original: result.getRight() + replicatedSize.addAndGet(sizes.getLeft()); // Original: result.getLeft() + count.addAndGet(repeatedOmKeyInfo.getOmKeyInfoList().size()); + } + } + return null; + }); + } + + return Triple.of(count.get(), unReplicatedSize.get(), replicatedSize.get()); + } + @Override public String getTaskName() { return "OmTableInsightTask"; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java index 0640d4ad30b0..c380f102687c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -202,6 +202,9 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, } waitForQueueSize(iterFutures, 0); waitForQueueSize(workerFutures, 0); + + // Log final stats + LOG.info("{}: Parallel iteration completed - Total keys processed: {}", taskName, keyCounter.get()); } @Override From 7cfc00099e7d836c575a99d10ffa370317ccadba Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 10 Nov 2025 12:11:31 +0530 Subject: [PATCH 05/49] Removed the commented code --- .../impl/OzoneManagerServiceProviderImpl.java | 280 +++++++++--------- 1 file changed, 136 insertions(+), 144 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 9751a63eae31..874049bfa89e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -114,7 +114,7 @@ @Singleton public class OzoneManagerServiceProviderImpl implements OzoneManagerServiceProvider { - private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); + private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; @@ -702,153 +702,145 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = true; - + boolean fullSnapshot = false; - if (reInitializeTasksCalled.compareAndSet(false, true)) { - LOG.info("Calling reprocess on Recon tasks."); - reconTaskController.reInitializeTasks(omMetadataManager, null); + if (currentSequenceNumber <= 0) { + fullSnapshot = true; } else { - LOG.info("reInitializeTasks already called once; skipping."); + // Get updates from OM and apply to local Recon OM DB and update task status in table + deltaReconTaskStatusUpdater.recordRunStart(); + int loopCount = 0; + long fromSequenceNumber = currentSequenceNumber; + long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; + /** + * This loop will continue to fetch and apply OM DB updates and with every + * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. + * It continues to fetch from OM till the lag, between OM DB WAL sequence number + * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. + * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. + */ + while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { + try (OMDBUpdatesHandler omdbUpdatesHandler = + new OMDBUpdatesHandler(omMetadataManager)) { + + // If interrupt was previously signalled, + // we should check for it before starting delta update sync. + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted during delta update."); + } + diffBetweenOMDbAndReconDBSeqNumber = + getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); + deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); + // Keeping last updated sequence number for both full and delta tasks to be same + // because sequence number of DB denotes and points to same OM DB copy of Recon, + // even though two different tasks are updating the DB at different conditions, but + // it tells the sync state with actual OM DB for the same Recon OM DB copy. + deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + deltaReconTaskStatusUpdater.recordRunCompletion(); + fullSnapshotReconTaskUpdater.updateDetails(); + // Update the current OM metadata manager in task controller + reconTaskController.updateOMMetadataManager(omMetadataManager); + + // Pass on DB update events to tasks that are listening. + reconTaskController.consumeOMEvents(new OMUpdateEventBatch( + omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); + + // Check if task reinitialization is needed due to buffer overflow or task failures + boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); + boolean tasksFailed = reconTaskController.hasTasksFailed(); + + if (bufferOverflowed || tasksFailed) { + ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? + ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : + ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; + + LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", + reason); + + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + + // Queue async reinitialization event - checkpoint creation and retry logic is handled internally + ReconTaskController.ReInitializationResult result = + reconTaskController.queueReInitializationEvent(reason); + + //TODO: Create a metric to track this event buffer overflow or task failure event + boolean triggerFullSnapshot = + Optional.ofNullable(result) + .map(r -> { + switch (r) { + case MAX_RETRIES_EXCEEDED: + LOG.warn( + "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + + "fallback"); + return true; + + case RETRY_LATER: + LOG.debug("Reinitialization event queueing will be retried in next iteration"); + return false; + + default: + LOG.info("Reinitialization event successfully queued"); + return false; + } + }) + .orElseGet(() -> { + LOG.error( + "ReInitializationResult is null, something went wrong in queueing reinitialization " + + "event"); + return true; + }); + + if (triggerFullSnapshot) { + fullSnapshot = true; + } + } + currentSequenceNumber = getCurrentOMDBSequenceNumber(); + LOG.debug("Updated current sequence number: {}", currentSequenceNumber); + loopCount++; + } catch (InterruptedException intEx) { + LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); + // We are updating the table even if it didn't run i.e. got interrupted beforehand + // to indicate that a task was supposed to run, but it didn't. + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + Thread.currentThread().interrupt(); + // Since thread is interrupted, we do not fall back to snapshot sync. + // Return with sync failed status. + return false; + } catch (Exception e) { + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", + e.getMessage()); + fullSnapshot = true; + } + if (fullSnapshot) { + break; + } + } + LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, + getCurrentOMDBSequenceNumber() - fromSequenceNumber); } -// if (currentSequenceNumber <= 0) { -// fullSnapshot = true; -// } else { -// // Get updates from OM and apply to local Recon OM DB and update task status in table -// deltaReconTaskStatusUpdater.recordRunStart(); -// int loopCount = 0; -// long fromSequenceNumber = currentSequenceNumber; -// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; -// /** -// * This loop will continue to fetch and apply OM DB updates and with every -// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. -// * It continues to fetch from OM till the lag, between OM DB WAL sequence number -// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. -// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. -// */ -// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { -// try (OMDBUpdatesHandler omdbUpdatesHandler = -// new OMDBUpdatesHandler(omMetadataManager)) { -// -// // If interrupt was previously signalled, -// // we should check for it before starting delta update sync. -// if (Thread.currentThread().isInterrupted()) { -// throw new InterruptedException("Thread interrupted during delta update."); -// } -// diffBetweenOMDbAndReconDBSeqNumber = -// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); -// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); -// // Keeping last updated sequence number for both full and delta tasks to be same -// // because sequence number of DB denotes and points to same OM DB copy of Recon, -// // even though two different tasks are updating the DB at different conditions, but -// // it tells the sync state with actual OM DB for the same Recon OM DB copy. -// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// deltaReconTaskStatusUpdater.recordRunCompletion(); -// fullSnapshotReconTaskUpdater.updateDetails(); -// // Update the current OM metadata manager in task controller -// reconTaskController.updateOMMetadataManager(omMetadataManager); -// -// // Pass on DB update events to tasks that are listening. -// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( -// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); -// -// // Check if task reinitialization is needed due to buffer overflow or task failures -// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); -// boolean tasksFailed = reconTaskController.hasTasksFailed(); -// -// if (bufferOverflowed || tasksFailed) { -// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? -// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : -// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; -// -// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", -// reason); -// -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// -// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally -// ReconTaskController.ReInitializationResult result = -// reconTaskController.queueReInitializationEvent(reason); -// -// //TODO: Create a metric to track this event buffer overflow or task failure event -// boolean triggerFullSnapshot = -// Optional.ofNullable(result) -// .map(r -> { -// switch (r) { -// case MAX_RETRIES_EXCEEDED: -// LOG.warn( -// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + -// "fallback"); -// return true; -// -// case RETRY_LATER: -// LOG.debug("Reinitialization event queueing will be retried in next iteration"); -// return false; -// -// default: -// LOG.info("Reinitialization event successfully queued"); -// return false; -// } -// }) -// .orElseGet(() -> { -// LOG.error( -// "ReInitializationResult is null, something went wrong in queueing reinitialization " + -// "event"); -// return true; -// }); -// -// if (triggerFullSnapshot) { -// fullSnapshot = true; -// } -// } -// currentSequenceNumber = getCurrentOMDBSequenceNumber(); -// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); -// loopCount++; -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); -// // We are updating the table even if it didn't run i.e. got interrupted beforehand -// // to indicate that a task was supposed to run, but it didn't. -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// Thread.currentThread().interrupt(); -// // Since thread is interrupted, we do not fall back to snapshot sync. -// // Return with sync failed status. -// return false; -// } catch (Exception e) { -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", -// e.getMessage()); -// fullSnapshot = true; -// } -// if (fullSnapshot) { -// break; -// } -// } -// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, -// getCurrentOMDBSequenceNumber() - fromSequenceNumber); -// } - -// if (fullSnapshot) { -// try { -// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Snapshot update sync thread was interrupted."); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// Thread.currentThread().interrupt(); -// // Mark sync status as failed. -// return false; -// } catch (Exception e) { -// metrics.incrNumSnapshotRequestsFailed(); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); -// // Update health status in ReconContext -// reconContext.updateHealthStatus(new AtomicBoolean(false)); -// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); -// } -// } + if (fullSnapshot) { + try { + executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); + } catch (InterruptedException intEx) { + LOG.error("OM DB Snapshot update sync thread was interrupted."); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + Thread.currentThread().interrupt(); + // Mark sync status as failed. + return false; + } catch (Exception e) { + metrics.incrNumSnapshotRequestsFailed(); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + LOG.error("Unable to update Recon's metadata with new OM DB. ", e); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); + } + } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); From e67b3101b1c10ca61527a0362407bf63d3c59b2e Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 10 Nov 2025 13:23:02 +0530 Subject: [PATCH 06/49] Made code review changes- Added RejectionPolicy, Re-added flush logic for fileSizeCountTask, and made ReentrantReadWriteLock fair --- .../recon/tasks/ContainerKeyMapperHelper.java | 3 +- .../recon/tasks/FileSizeCountTaskHelper.java | 31 +++++++++++++++++-- .../util/ParallelTableIteratorOperation.java | 18 +++++++++-- 3 files changed, 47 insertions(+), 5 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 413fa2fa4471..81559a620639 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -110,7 +110,8 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Get the appropriate table based on BucketLayout Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + // Use fair lock to prevent write lock starvation when flushing + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); // Use parallel table iteration Function, Void> kvOperation = kv -> { try { diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index 78bdd7005467..d6ca8b5b601c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.hdds.utils.db.StringCodec; @@ -161,15 +162,41 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); long startTime = Time.monotonicNow(); + // Use fair lock to prevent write lock starvation when flushing + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + final int FLUSH_THRESHOLD = 100000; + // Use parallel table iteration Function, Void> kvOperation = kv -> { - handlePutKeyEvent(kv.getValue(), fileSizeCountMap); + try { + lock.readLock().lock(); + handlePutKeyEvent(kv.getValue(), fileSizeCountMap); + } finally { + lock.readLock().unlock(); + } + + // Periodic flush to prevent unbounded memory growth + if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { + try { + lock.writeLock().lock(); + // 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); + fileSizeCountMap.clear(); + } + } catch (Exception e) { + throw new RuntimeException("Failed to flush file size counts", e); + } finally { + lock.writeLock().unlock(); + } + } return null; }; try (ParallelTableIteratorOperation keyIter = new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, 100000)) { + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, FLUSH_THRESHOLD)) { keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } catch (Exception ex) { LOG.error("Unable to populate File Size Count for {} in RocksDB.", taskName, ex); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java index c380f102687c..ca4bbf6cfc4f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -71,9 +71,11 @@ public ParallelTableIteratorOperation(OMMetadataManager metadataManager, Table(iteratorCount * 2)); + new ArrayBlockingQueue<>(iteratorCount * 2), + new ThreadPoolExecutor.CallerRunsPolicy()); this.valueExecutors = new ThreadPoolExecutor(workerCount, workerCount, 1, TimeUnit.MINUTES, - new ArrayBlockingQueue<>(workerCount * 2)); + new ArrayBlockingQueue<>(workerCount * 2), + new ThreadPoolExecutor.CallerRunsPolicy()); this.maxNumberOfVals = Math.max(10, maxNumberOfValsInMemory / (workerCount)); this.logCountThreshold = logThreshold; } @@ -211,6 +213,18 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, public void close() throws IOException { iteratorExecutor.shutdown(); valueExecutors.shutdown(); + try { + if (!iteratorExecutor.awaitTermination(60, TimeUnit.SECONDS)) { + iteratorExecutor.shutdownNow(); + } + if (!valueExecutors.awaitTermination(60, TimeUnit.SECONDS)) { + valueExecutors.shutdownNow(); + } + } catch (InterruptedException e) { + iteratorExecutor.shutdownNow(); + valueExecutors.shutdownNow(); + Thread.currentThread().interrupt(); + } } } From a9855b095360abb2d4a1d4d7e4987bd1705517a0 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 10 Nov 2025 13:44:07 +0530 Subject: [PATCH 07/49] Added a flag isFlushingInProgress to avoid race condition while flushing --- .../recon/tasks/ContainerKeyMapperHelper.java | 19 ++++++++++++---- .../recon/tasks/FileSizeCountTaskHelper.java | 22 +++++++++++++------ 2 files changed, 30 insertions(+), 11 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 81559a620639..fdf9ab573bea 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; @@ -112,6 +113,9 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // 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, Void> kvOperation = kv -> { try { @@ -123,12 +127,19 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, lock.readLock().unlock(); } omKeyCount.incrementAndGet(); - if (containerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { + + // Only one thread should attempt flush to avoid blocking all workers + if (containerKeyMap.size() >= containerKeyFlushToDBMaxThreshold && + isFlushingInProgress.compareAndSet(false, true)) { try { lock.writeLock().lock(); - if (!checkAndCallFlushToDB(containerKeyMap, containerKeyFlushToDBMaxThreshold, - reconContainerMetadataManager)) { - throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); + try { + if (!checkAndCallFlushToDB(containerKeyMap, containerKeyFlushToDBMaxThreshold, + reconContainerMetadataManager)) { + throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); + } + } finally { + isFlushingInProgress.set(false); // Reset flag after flush completes } } finally { lock.writeLock().unlock(); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index d6ca8b5b601c..89e93925c42a 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; @@ -164,6 +165,8 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, // 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 @@ -175,15 +178,20 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, lock.readLock().unlock(); } - // Periodic flush to prevent unbounded memory growth - if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { + // Only one thread should attempt flush to avoid blocking all workers + if (fileSizeCountMap.size() >= FLUSH_THRESHOLD && + isFlushingInProgress.compareAndSet(false, true)) { try { lock.writeLock().lock(); - // 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); - fileSizeCountMap.clear(); + 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); + fileSizeCountMap.clear(); + } + } finally { + isFlushingInProgress.set(false); // Reset flag after flush completes } } catch (Exception e) { throw new RuntimeException("Failed to flush file size counts", e); From 56a2e40bf70bf9e0b8a88475cb66ef5a4ddb6bf8 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 10 Nov 2025 18:47:08 +0530 Subject: [PATCH 08/49] Made final code changes for testing --- .../util/ParallelTableIteratorOperation.java | 34 ++++++++++++++----- .../recon/tasks/TestFileSizeCountTask.java | 2 ++ 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java index ca4bbf6cfc4f..d85af159d5e6 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -82,18 +82,34 @@ public ParallelTableIteratorOperation(OMMetadataManager metadataManager, Table getBounds(K startKey, K endKey) throws IOException { - RDBStore store = (RDBStore) this.metadataManager.getStore(); - List sstFiles = store.getDb().getSstFileList(); Set keys = new HashSet<>(); - String tableName = table.getName(); - 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())); + + // 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 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); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java index ca4a0399be80..c4b1041f76af 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java @@ -134,6 +134,7 @@ public void testReprocess() throws IOException { // Note: Even though legacy and OBS share the same underlying table, we simulate OBS here. when(omMetadataManager.getKeyTable(eq(BucketLayout.OBJECT_STORE))) .thenReturn(keyTableOBS); + when(keyTableOBS.getName()).thenReturn("keyTable"); // Mock table name for parallelization TypedTable.TypedTableIterator mockIterOBS = mock(TypedTable.TypedTableIterator.class); when(keyTableOBS.iterator()).thenReturn(mockIterOBS); // Simulate three keys then end. @@ -146,6 +147,7 @@ public void testReprocess() throws IOException { TypedTable keyTableFSO = mock(TypedTable.class); when(omMetadataManager.getKeyTable(eq(BucketLayout.FILE_SYSTEM_OPTIMIZED))) .thenReturn(keyTableFSO); + when(keyTableFSO.getName()).thenReturn("fileTable"); // Mock table name for parallelization TypedTable.TypedTableIterator mockIterFSO = mock(TypedTable.TypedTableIterator.class); when(keyTableFSO.iterator()).thenReturn(mockIterFSO); when(mockIterFSO.hasNext()).thenReturn(true, true, true, false); From 8424170564024b22404c06b53356afd392d89398 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 12 Nov 2025 13:44:40 +0530 Subject: [PATCH 09/49] Prevent FSO/OBS container count overwrite by using a shared ConcurrentHashMap for container counts --- .../hadoop/ozone/recon/ReconConstants.java | 3 + .../impl/OzoneManagerServiceProviderImpl.java | 304 +++++++++--------- .../recon/tasks/ContainerKeyMapperHelper.java | 162 ++++++---- .../recon/tasks/FileSizeCountTaskHelper.java | 2 +- .../tasks/TestContainerKeyMapperTask.java | 5 + 5 files changed, 269 insertions(+), 207 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java index 396beb9e3fa3..890253839ec7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java @@ -95,6 +95,8 @@ public final class ReconConstants { public static final AtomicBoolean CONTAINER_KEY_TABLES_TRUNCATED = new AtomicBoolean(false); + public static final AtomicBoolean CONTAINER_KEY_COUNT_MAP_INITIALIZED = new AtomicBoolean(false); + private ReconConstants() { // Never Constructed } @@ -106,5 +108,6 @@ private ReconConstants() { public static void resetTableTruncatedFlags() { FILE_SIZE_COUNT_TABLE_TRUNCATED.set(false); CONTAINER_KEY_TABLES_TRUNCATED.set(false); + CONTAINER_KEY_COUNT_MAP_INITIALIZED.set(false); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 874049bfa89e..0b5a9a9e92bb 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -114,7 +114,7 @@ @Singleton public class OzoneManagerServiceProviderImpl implements OzoneManagerServiceProvider { - + private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; @@ -346,17 +346,17 @@ private void startSyncDataFromOM(long initialDelay) { TimeUnit.MILLISECONDS); LOG.debug("Started the OM DB sync scheduler."); scheduler.scheduleWithFixedDelay(() -> { - try { - LOG.info("Last known sequence number before sync: {}", getCurrentOMDBSequenceNumber()); - boolean isSuccess = syncDataFromOM(); - if (!isSuccess) { - LOG.debug("OM DB sync is already running, or encountered an error while trying to sync data."); - } - LOG.info("Sequence number after sync: {}", getCurrentOMDBSequenceNumber()); - } catch (Throwable t) { - LOG.error("Unexpected exception while syncing data from OM.", t); - } - }, + try { + LOG.info("Last known sequence number before sync: {}", getCurrentOMDBSequenceNumber()); + boolean isSuccess = syncDataFromOM(); + if (!isSuccess) { + LOG.debug("OM DB sync is already running, or encountered an error while trying to sync data."); + } + LOG.info("Sequence number after sync: {}", getCurrentOMDBSequenceNumber()); + } catch (Throwable t) { + LOG.error("Unexpected exception while syncing data from OM.", t); + } + }, initialDelay, interval, TimeUnit.MILLISECONDS); @@ -702,145 +702,153 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = false; + boolean fullSnapshot = true; + - if (currentSequenceNumber <= 0) { - fullSnapshot = true; + if (reInitializeTasksCalled.compareAndSet(false, true)) { + LOG.info("Calling reprocess on Recon tasks."); + reconTaskController.reInitializeTasks(omMetadataManager, null); } else { - // Get updates from OM and apply to local Recon OM DB and update task status in table - deltaReconTaskStatusUpdater.recordRunStart(); - int loopCount = 0; - long fromSequenceNumber = currentSequenceNumber; - long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; - /** - * This loop will continue to fetch and apply OM DB updates and with every - * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. - * It continues to fetch from OM till the lag, between OM DB WAL sequence number - * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. - * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. - */ - while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { - try (OMDBUpdatesHandler omdbUpdatesHandler = - new OMDBUpdatesHandler(omMetadataManager)) { - - // If interrupt was previously signalled, - // we should check for it before starting delta update sync. - if (Thread.currentThread().isInterrupted()) { - throw new InterruptedException("Thread interrupted during delta update."); - } - diffBetweenOMDbAndReconDBSeqNumber = - getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); - deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); - // Keeping last updated sequence number for both full and delta tasks to be same - // because sequence number of DB denotes and points to same OM DB copy of Recon, - // even though two different tasks are updating the DB at different conditions, but - // it tells the sync state with actual OM DB for the same Recon OM DB copy. - deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - deltaReconTaskStatusUpdater.recordRunCompletion(); - fullSnapshotReconTaskUpdater.updateDetails(); - // Update the current OM metadata manager in task controller - reconTaskController.updateOMMetadataManager(omMetadataManager); - - // Pass on DB update events to tasks that are listening. - reconTaskController.consumeOMEvents(new OMUpdateEventBatch( - omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); - - // Check if task reinitialization is needed due to buffer overflow or task failures - boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); - boolean tasksFailed = reconTaskController.hasTasksFailed(); - - if (bufferOverflowed || tasksFailed) { - ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? - ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : - ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; - - LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", - reason); - - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - - // Queue async reinitialization event - checkpoint creation and retry logic is handled internally - ReconTaskController.ReInitializationResult result = - reconTaskController.queueReInitializationEvent(reason); - - //TODO: Create a metric to track this event buffer overflow or task failure event - boolean triggerFullSnapshot = - Optional.ofNullable(result) - .map(r -> { - switch (r) { - case MAX_RETRIES_EXCEEDED: - LOG.warn( - "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + - "fallback"); - return true; - - case RETRY_LATER: - LOG.debug("Reinitialization event queueing will be retried in next iteration"); - return false; - - default: - LOG.info("Reinitialization event successfully queued"); - return false; - } - }) - .orElseGet(() -> { - LOG.error( - "ReInitializationResult is null, something went wrong in queueing reinitialization " + - "event"); - return true; - }); - - if (triggerFullSnapshot) { - fullSnapshot = true; - } - } - currentSequenceNumber = getCurrentOMDBSequenceNumber(); - LOG.debug("Updated current sequence number: {}", currentSequenceNumber); - loopCount++; - } catch (InterruptedException intEx) { - LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); - // We are updating the table even if it didn't run i.e. got interrupted beforehand - // to indicate that a task was supposed to run, but it didn't. - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - Thread.currentThread().interrupt(); - // Since thread is interrupted, we do not fall back to snapshot sync. - // Return with sync failed status. - return false; - } catch (Exception e) { - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", - e.getMessage()); - fullSnapshot = true; - } - if (fullSnapshot) { - break; - } - } - LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, - getCurrentOMDBSequenceNumber() - fromSequenceNumber); + LOG.info("reInitializeTasks already called once; skipping."); } - if (fullSnapshot) { - try { - executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); - } catch (InterruptedException intEx) { - LOG.error("OM DB Snapshot update sync thread was interrupted."); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - Thread.currentThread().interrupt(); - // Mark sync status as failed. - return false; - } catch (Exception e) { - metrics.incrNumSnapshotRequestsFailed(); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - LOG.error("Unable to update Recon's metadata with new OM DB. ", e); - // Update health status in ReconContext - reconContext.updateHealthStatus(new AtomicBoolean(false)); - reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); - } - } +// if (currentSequenceNumber <= 0) { +// fullSnapshot = true; +// } else { +// // Get updates from OM and apply to local Recon OM DB and update task status in table +// deltaReconTaskStatusUpdater.recordRunStart(); +// int loopCount = 0; +// long fromSequenceNumber = currentSequenceNumber; +// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; +// /** +// * This loop will continue to fetch and apply OM DB updates and with every +// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. +// * It continues to fetch from OM till the lag, between OM DB WAL sequence number +// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. +// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. +// */ +// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { +// try (OMDBUpdatesHandler omdbUpdatesHandler = +// new OMDBUpdatesHandler(omMetadataManager)) { +// +// // If interrupt was previously signalled, +// // we should check for it before starting delta update sync. +// if (Thread.currentThread().isInterrupted()) { +// throw new InterruptedException("Thread interrupted during delta update."); +// } +// diffBetweenOMDbAndReconDBSeqNumber = +// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); +// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); +// // Keeping last updated sequence number for both full and delta tasks to be same +// // because sequence number of DB denotes and points to same OM DB copy of Recon, +// // even though two different tasks are updating the DB at different conditions, but +// // it tells the sync state with actual OM DB for the same Recon OM DB copy. +// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// deltaReconTaskStatusUpdater.recordRunCompletion(); +// fullSnapshotReconTaskUpdater.updateDetails(); +// // Update the current OM metadata manager in task controller +// reconTaskController.updateOMMetadataManager(omMetadataManager); +// +// // Pass on DB update events to tasks that are listening. +// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( +// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); +// +// // Check if task reinitialization is needed due to buffer overflow or task failures +// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); +// boolean tasksFailed = reconTaskController.hasTasksFailed(); +// +// if (bufferOverflowed || tasksFailed) { +// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? +// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : +// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; +// +// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", +// reason); +// +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// +// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally +// ReconTaskController.ReInitializationResult result = +// reconTaskController.queueReInitializationEvent(reason); +// +// //TODO: Create a metric to track this event buffer overflow or task failure event +// boolean triggerFullSnapshot = +// Optional.ofNullable(result) +// .map(r -> { +// switch (r) { +// case MAX_RETRIES_EXCEEDED: +// LOG.warn( +// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + +// "fallback"); +// return true; +// +// case RETRY_LATER: +// LOG.debug("Reinitialization event queueing will be retried in next iteration"); +// return false; +// +// default: +// LOG.info("Reinitialization event successfully queued"); +// return false; +// } +// }) +// .orElseGet(() -> { +// LOG.error( +// "ReInitializationResult is null, something went wrong in queueing reinitialization " + +// "event"); +// return true; +// }); +// +// if (triggerFullSnapshot) { +// fullSnapshot = true; +// } +// } +// currentSequenceNumber = getCurrentOMDBSequenceNumber(); +// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); +// loopCount++; +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); +// // We are updating the table even if it didn't run i.e. got interrupted beforehand +// // to indicate that a task was supposed to run, but it didn't. +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// Thread.currentThread().interrupt(); +// // Since thread is interrupted, we do not fall back to snapshot sync. +// // Return with sync failed status. +// return false; +// } catch (Exception e) { +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", +// e.getMessage()); +// fullSnapshot = true; +// } +// if (fullSnapshot) { +// break; +// } +// } +// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, +// getCurrentOMDBSequenceNumber() - fromSequenceNumber); +// } + +// if (fullSnapshot) { +// try { +// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Snapshot update sync thread was interrupted."); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// Thread.currentThread().interrupt(); +// // Mark sync status as failed. +// return false; +// } catch (Exception e) { +// metrics.incrNumSnapshotRequestsFailed(); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); +// // Update health status in ReconContext +// reconContext.updateHealthStatus(new AtomicBoolean(false)); +// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); +// } +// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); @@ -859,7 +867,7 @@ private void markDeltaTaskStatusAsFailed(ReconTaskStatusUpdater deltaReconTaskSt } private void executeFullSnapshot(ReconTaskStatusUpdater fullSnapshotReconTaskUpdater, - ReconTaskStatusUpdater deltaReconTaskStatusUpdater) throws InterruptedException, IOException { + ReconTaskStatusUpdater deltaReconTaskStatusUpdater) throws InterruptedException, IOException { metrics.incrNumSnapshotRequests(); LOG.info("Obtaining full snapshot from Ozone Manager"); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index fdf9ab573bea..c0e163188611 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -62,6 +62,18 @@ public abstract class ContainerKeyMapperHelper { // Static lock to guard table truncation. private static final Object TRUNCATE_LOCK = new Object(); + /** + * SHARED across all tasks (FSO + OBS) for cross-task synchronization. + * Maps: ContainerId -> AtomicLong (key count in that container) + * Purpose: Prevents data corruption when FSO and OBS tasks run concurrently + * and both write to the same container IDs. Both tasks accumulate into this + * single shared map, ensuring final DB write contains complete totals. + */ + private static final Map SHARED_CONTAINER_KEY_COUNT_MAP = new ConcurrentHashMap<>(); + + // Lock to guard shared map initialization/clearing + private static final Object SHARED_MAP_LOCK = new Object(); + /** * Ensures that the container key tables are truncated only once before reprocessing. * Uses an AtomicBoolean to track if truncation has already been performed. @@ -88,6 +100,33 @@ public static void truncateTablesIfNeeded(ReconContainerMetadataManager reconCon } } + /** + * Ensures the shared container count map is cleared once per reprocess cycle. + * This must be called by the first task that starts reprocessing to prevent + * cross-task data corruption where FSO and OBS tasks overwrite each other's counts. + */ + private static void initializeSharedContainerCountMapIfNeeded(String taskName) { + synchronized (SHARED_MAP_LOCK) { + if (ReconConstants.CONTAINER_KEY_COUNT_MAP_INITIALIZED.compareAndSet(false, true)) { + SHARED_CONTAINER_KEY_COUNT_MAP.clear(); + LOG.info("{}: Initialized shared container count map for cross-task synchronization", taskName); + } else { + LOG.debug("{}: Shared container count map already initialized by another task", taskName); + } + } + } + + /** + * Clears the shared container count map and resets its initialization flag. + * This method should be called by tests to ensure clean state between test runs. + */ + public static void clearSharedContainerCountMap() { + synchronized (SHARED_MAP_LOCK) { + SHARED_CONTAINER_KEY_COUNT_MAP.clear(); + ReconConstants.CONTAINER_KEY_COUNT_MAP_INITIALIZED.set(false); + } + } + public static boolean reprocess(OMMetadataManager omMetadataManager, ReconContainerMetadataManager reconContainerMetadataManager, BucketLayout bucketLayout, @@ -97,8 +136,9 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, int maxWorkers, int maxKeysInMemory) { AtomicLong omKeyCount = new AtomicLong(0); - Map containerKeyMap = new ConcurrentHashMap<>(); - Map containerKeyCountMap = new ConcurrentHashMap<>(); + // Local map: per-task ContainerKeyPrefix mappings (cleared on flush) + Map localContainerKeyMap = new ConcurrentHashMap<>(); + // Shared map: cross-task container counts (FSO + OBS accumulate here) try { LOG.info("{}: Starting parallel reprocess with {} iterators, {} workers, max {} keys in memory for bucket layout {}", @@ -108,11 +148,14 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Ensure the tables are truncated only once truncateTablesIfNeeded(reconContainerMetadataManager, taskName); + // Initialize shared container count map once per cycle + initializeSharedContainerCountMapIfNeeded(taskName); + // Get the appropriate table based on BucketLayout Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); // Use fair lock to prevent write lock starvation when flushing - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); // Flag to coordinate flush attempts - prevents all threads from queuing for write lock AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); @@ -121,7 +164,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, try { try { lock.readLock().lock(); - handleKeyReprocess(kv.getKey(), kv.getValue(), containerKeyMap, containerKeyCountMap, + handleKeyReprocess(kv.getKey(), kv.getValue(), localContainerKeyMap, SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager); } finally { lock.readLock().unlock(); @@ -129,12 +172,12 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, omKeyCount.incrementAndGet(); // Only one thread should attempt flush to avoid blocking all workers - if (containerKeyMap.size() >= containerKeyFlushToDBMaxThreshold && + if (localContainerKeyMap.size() >= containerKeyFlushToDBMaxThreshold && isFlushingInProgress.compareAndSet(false, true)) { try { lock.writeLock().lock(); try { - if (!checkAndCallFlushToDB(containerKeyMap, containerKeyFlushToDBMaxThreshold, + if (!checkAndCallFlushToDB(localContainerKeyMap, containerKeyFlushToDBMaxThreshold, reconContainerMetadataManager)) { throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); } @@ -156,8 +199,8 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } - // Final flush and commit - if (!flushAndCommitContainerKeyInfoToDB(containerKeyMap, containerKeyCountMap, reconContainerMetadataManager)) { + // Final flush and commit: local map cleared, shared map kept for other tasks + if (!flushAndCommitContainerKeyInfoToDB(localContainerKeyMap, SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager)) { LOG.error("Failed to flush Container Key data to DB for {}", taskName); return false; } @@ -169,9 +212,9 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, double throughput = keysProcessed / Math.max(durationSeconds, 0.001); LOG.info("{}: Parallel reprocess completed. Processed {} keys in {} ms ({} sec) - " + - "Throughput: {} keys/sec - Containers: {}, Container-Key mappings: {}", + "Throughput: {} keys/sec - Containers (shared): {}, Container-Key mappings (local): {}", taskName, keysProcessed, durationMillis, String.format("%.2f", durationSeconds), - String.format("%.2f", throughput), containerKeyCountMap.size(), containerKeyMap.size()); + String.format("%.2f", throughput), SHARED_CONTAINER_KEY_COUNT_MAP.size(), localContainerKeyMap.size()); } catch (Exception ex) { LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ex); return false; @@ -179,11 +222,11 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, return true; } - private static synchronized boolean checkAndCallFlushToDB(Map containerKeyMap, + private static synchronized boolean checkAndCallFlushToDB(Map localContainerKeyMap, long containerKeyFlushToDBMaxThreshold, ReconContainerMetadataManager reconContainerMetadataManager) { - if (containerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { - return flushAndCommitContainerKeyInfoToDB(containerKeyMap, Collections.emptyMap(), reconContainerMetadataManager); + if (localContainerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { + return flushAndCommitContainerKeyInfoToDB(localContainerKeyMap, Collections.emptyMap(), reconContainerMetadataManager); } return true; } @@ -196,14 +239,14 @@ public static boolean process(OMUpdateEventBatch events, int eventCount = 0; // In-memory maps for fast look up and batch write - // (HDDS-8580) containerKeyMap map is allowed to be used + // (HDDS-8580) localContainerKeyMap map is allowed to be used // in "process" without batching since the maximum number of keys // is bounded by delta limit configurations - // (container, key) -> count - Map containerKeyMap = new HashMap<>(); - // containerId -> key count - Map containerKeyCountMap = new HashMap<>(); + // Local map: (container, key) -> count (per event batch) + Map localContainerKeyMap = new HashMap<>(); + // Local map: containerId -> key count (per event batch) + Map localContainerKeyCountMap = new HashMap<>(); // List of the deleted (container, key) pair's List deletedKeyCountList = new ArrayList<>(); long startTime = Time.monotonicNow(); @@ -219,25 +262,25 @@ public static boolean process(OMUpdateEventBatch events, try { switch (omdbUpdateEvent.getAction()) { case PUT: - handlePutOMKeyEvent(updatedKey, updatedKeyValue, containerKeyMap, - containerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); + handlePutOMKeyEvent(updatedKey, updatedKeyValue, localContainerKeyMap, + localContainerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); break; case DELETE: - handleDeleteOMKeyEvent(updatedKey, containerKeyMap, - containerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); + handleDeleteOMKeyEvent(updatedKey, localContainerKeyMap, + localContainerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); break; case UPDATE: if (omdbUpdateEvent.getOldValue() != null) { handleDeleteOMKeyEvent( - omdbUpdateEvent.getOldValue().getKeyName(), containerKeyMap, - containerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); + omdbUpdateEvent.getOldValue().getKeyName(), localContainerKeyMap, + localContainerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); } else { LOG.warn("Update event does not have the old Key Info for {}.", updatedKey); } - handlePutOMKeyEvent(updatedKey, updatedKeyValue, containerKeyMap, - containerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); + handlePutOMKeyEvent(updatedKey, updatedKeyValue, localContainerKeyMap, + localContainerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); break; default: @@ -250,7 +293,10 @@ public static boolean process(OMUpdateEventBatch events, } } try { - writeToTheDB(containerKeyMap, containerKeyCountMap, deletedKeyCountList, reconContainerMetadataManager); + // Convert local Long map to AtomicLong map for writeToTheDB compatibility + Map localContainerKeyCountMapAtomic = new ConcurrentHashMap<>(); + localContainerKeyCountMap.forEach((k, v) -> localContainerKeyCountMapAtomic.put(k, new AtomicLong(v))); + writeToTheDB(localContainerKeyMap, localContainerKeyCountMapAtomic, deletedKeyCountList, reconContainerMetadataManager); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); return false; @@ -385,29 +431,30 @@ private static void handleDeleteOMKeyEvent(String key, } } - private static void writeToTheDB(Map containerKeyMap, - Map containerKeyCountMap, + private static void writeToTheDB(Map localContainerKeyMap, + Map containerKeyCountMap, List deletedContainerKeyList, ReconContainerMetadataManager reconContainerMetadataManager) throws IOException { try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { - // Write container key mappings - containerKeyMap.keySet().forEach((ContainerKeyPrefix key) -> { + // Write container key mappings (local per-task data) + localContainerKeyMap.keySet().forEach((ContainerKeyPrefix key) -> { try { reconContainerMetadataManager.batchStoreContainerKeyMapping( - rdbBatchOperation, key, containerKeyMap.get(key)); + rdbBatchOperation, key, localContainerKeyMap.get(key)); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); } }); - // Write container key count mappings + // Write container key count mappings (can be local or shared depending on caller) containerKeyCountMap.keySet().forEach((Long key) -> { try { + long count = containerKeyCountMap.get(key).get(); // Get value from AtomicLong reconContainerMetadataManager.batchStoreContainerKeyCounts( - rdbBatchOperation, key, containerKeyCountMap.get(key)); + rdbBatchOperation, key, count); } catch (IOException e) { LOG.error("Unable to write Container Key Count data in Recon DB.", e); } @@ -434,18 +481,17 @@ private static void writeToTheDB(Map containerKeyMa * * @param key key String * @param omKeyInfo omKeyInfo value - * @param containerKeyMap we keep the added containerKeys in this map - * to allow incremental batching to containerKeyTable - * @param containerKeyCountMap we keep the containerKey counts in this map - * to allow batching to containerKeyCountTable - * after reprocessing is done + * @param localContainerKeyMap Local per-task map for ContainerKeyPrefix mappings + * (cleared on flush, not shared between tasks) + * @param sharedContainerKeyCountMap Shared cross-task map for container counts + * (FSO + OBS both update this, uses AtomicLong for thread safety) * @param reconContainerMetadataManager Recon metadata manager instance * @throws IOException if unable to write to recon DB. */ public static void handleKeyReprocess(String key, OmKeyInfo omKeyInfo, - Map containerKeyMap, - Map containerKeyCountMap, + Map localContainerKeyMap, + Map sharedContainerKeyCountMap, ReconContainerMetadataManager reconContainerMetadataManager) throws IOException { @@ -458,19 +504,20 @@ public static void handleKeyReprocess(String key, ContainerKeyPrefix containerKeyPrefix = ContainerKeyPrefix.get(containerId, key, keyVersion); if (reconContainerMetadataManager.getCountForContainerKeyPrefix(containerKeyPrefix) == 0 - && !containerKeyMap.containsKey(containerKeyPrefix)) { + && !localContainerKeyMap.containsKey(containerKeyPrefix)) { // Save on writes. No need to save same container-key prefix mapping again. - containerKeyMap.put(containerKeyPrefix, 1); + localContainerKeyMap.put(containerKeyPrefix, 1); - // if it exists, update the count of keys for the given containerID - // else, increment the count of containers and initialize keyCount - containerKeyCountMap.compute(containerId, (k, v) -> { - if (v == null) { - containerCountToIncrement.incrementAndGet(); - return 1L; - } - return v + 1L; - }); + // Thread-safe increment using shared AtomicLong map (cross-task safe: FSO + OBS) + AtomicLong count = sharedContainerKeyCountMap.computeIfAbsent(containerId, + k -> new AtomicLong(0)); + + long newCount = count.incrementAndGet(); + + // Check if this is the first key for this container (across all tasks) + if (newCount == 1) { + containerCountToIncrement.incrementAndGet(); + } } } } @@ -481,17 +528,16 @@ public static void handleKeyReprocess(String key, } public static boolean flushAndCommitContainerKeyInfoToDB( - Map containerKeyMap, - Map containerKeyCountMap, + Map localContainerKeyMap, + Map sharedContainerKeyCountMap, ReconContainerMetadataManager reconContainerMetadataManager) { try { // No deleted container list needed since "reprocess" only has put operations - writeToTheDB(containerKeyMap, containerKeyCountMap, Collections.emptyList(), reconContainerMetadataManager); + writeToTheDB(localContainerKeyMap, sharedContainerKeyCountMap, Collections.emptyList(), reconContainerMetadataManager); - // Clear in-memory maps after successful commit - containerKeyMap.clear(); - containerKeyCountMap.clear(); + // Only clear localContainerKeyMap (per-task), keep sharedContainerKeyCountMap for other tasks + localContainerKeyMap.clear(); } catch (IOException e) { LOG.error("Unable to write Container Key and Container Key Count data in Recon DB.", e); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index 89e93925c42a..feb2033e71f9 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -164,7 +164,7 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, long startTime = Time.monotonicNow(); // Use fair lock to prevent write lock starvation when flushing - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); // Flag to coordinate flush attempts - prevents all threads from queuing for write lock AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); final int FLUSH_THRESHOLD = 100000; diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java index 653de02cdbc2..ce8ab5ce2162 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java @@ -44,6 +44,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; +import org.apache.hadoop.ozone.recon.ReconConstants; import org.apache.hadoop.ozone.recon.ReconTestInjector; import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; @@ -95,6 +96,10 @@ public void setUp() throws Exception { .build(); reconContainerMetadataManager = reconTestInjector.getInstance(ReconContainerMetadataManager.class); + + // Clear shared container count map and reset flags for clean test state + ContainerKeyMapperHelper.clearSharedContainerCountMap(); + ReconConstants.resetTableTruncatedFlags(); } @Test From b93511edc143f0a0bd9d745f5fccb186b5c78b55 Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 13 Nov 2025 17:13:22 +0530 Subject: [PATCH 10/49] Added write lock in the writeCountsToDB method --- .../recon/tasks/FileSizeCountTaskHelper.java | 74 ++++++------------- 1 file changed, 23 insertions(+), 51 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index feb2033e71f9..aad299842063 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -17,8 +17,6 @@ package org.apache.hadoop.ozone.recon.tasks; -import java.io.IOException; -import java.io.UncheckedIOException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -29,7 +27,6 @@ import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.hdds.utils.db.StringCodec; 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.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; @@ -50,6 +47,14 @@ public abstract class FileSizeCountTaskHelper { // Static lock object for table truncation synchronization private static final Object TRUNCATE_LOCK = new Object(); + /** + * Static write lock for cross-task synchronization during DB writes. + * Ensures read-modify-write operations are atomic when FSO and OBS tasks + * flush concurrently to the same FileSizeCountKey bins. + */ + private static final ReentrantReadWriteLock FILE_COUNT_WRITE_LOCK = + new ReentrantReadWriteLock(); + /** * Increments the count for a given key on a PUT event. */ @@ -288,70 +293,37 @@ public static ReconOmTask.TaskResult processEvents(OMUpdateEventBatch events, /** * Writes the accumulated file size counts to RocksDB using ReconFileMetadataManager. */ - /** - * Checks if the file count table is empty by trying to get the first entry. - * This mimics the SQL Derby behavior of isFileCountBySizeTableEmpty(). - */ - private static boolean isFileCountTableEmpty(ReconFileMetadataManager reconFileMetadataManager) { - try (TableIterator> iterator = - reconFileMetadataManager.getFileCountTable().iterator()) { - return !iterator.hasNext(); - } catch (Exception e) { - LOG.warn("Error checking if file count table is empty, assuming not empty", e); - return false; - } - } - public static void writeCountsToDB(Map fileSizeCountMap, ReconFileMetadataManager reconFileMetadataManager) { if (fileSizeCountMap.isEmpty()) { return; } - - boolean isTableEmpty = isFileCountTableEmpty(reconFileMetadataManager); - - LOG.debug("writeCountsToDB: processing {} entries, isTableEmpty={}", - fileSizeCountMap.size(), isTableEmpty); + // Global write lock ensures atomic read-modify-write across concurrent FSO/OBS flushes + FILE_COUNT_WRITE_LOCK.writeLock().lock(); try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { for (Map.Entry entry : fileSizeCountMap.entrySet()) { FileSizeCountKey key = entry.getKey(); Long deltaCount = entry.getValue(); - - LOG.debug("Processing key: {}, deltaCount: {}", key, deltaCount); - - if (isTableEmpty) { - // Direct insert when table is empty (like SQL Derby reprocess behavior) - LOG.debug("Direct insert (table empty): key={}, deltaCount={}", key, deltaCount); - if (deltaCount > 0L) { - reconFileMetadataManager.batchStoreFileSizeCount(rdbBatchOperation, key, deltaCount); - LOG.debug("Storing key={} with deltaCount={}", key, deltaCount); - } - } else { - // Incremental update when table has data (like SQL Derby incremental behavior) - Long existingCount = reconFileMetadataManager.getFileSizeCount(key); - Long newCount = (existingCount != null ? existingCount : 0L) + deltaCount; - - LOG.debug("Incremental update: key={}, existingCount={}, deltaCount={}, newCount={}", - key, existingCount, deltaCount, newCount); - - if (newCount > 0L) { - reconFileMetadataManager.batchStoreFileSizeCount(rdbBatchOperation, key, newCount); - LOG.debug("Storing key={} with newCount={}", key, newCount); - } else if (existingCount != null) { - // Delete key if count becomes 0 or negative - reconFileMetadataManager.batchDeleteFileSizeCount(rdbBatchOperation, key); - LOG.debug("Deleting key={} as newCount={} <= 0", key, newCount); - } + + // Always do read-modify-write to handle concurrent task flushes correctly + Long existingCount = reconFileMetadataManager.getFileSizeCount(key); + Long newCount = (existingCount != null ? existingCount : 0L) + deltaCount; + + if (newCount > 0L) { + reconFileMetadataManager.batchStoreFileSizeCount(rdbBatchOperation, key, newCount); + } else if (existingCount != null) { + // Delete key if count becomes 0 or negative + reconFileMetadataManager.batchDeleteFileSizeCount(rdbBatchOperation, key); } } - - LOG.debug("Committing batch operation with {} operations", fileSizeCountMap.size()); + reconFileMetadataManager.commitBatchOperation(rdbBatchOperation); - LOG.debug("Batch operation committed successfully"); } catch (Exception e) { LOG.error("Error writing file size counts to RocksDB", e); throw new RuntimeException("Failed to write to RocksDB", e); + } finally { + FILE_COUNT_WRITE_LOCK.writeLock().unlock(); } } From e37bb1107ec3f005b33cbd8a3786d26007a889df Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 14 Nov 2025 10:34:52 +0530 Subject: [PATCH 11/49] Removed extra locking and improved exception log messages --- .../ozone/recon/tasks/ContainerKeyMapperHelper.java | 2 +- .../tasks/util/ParallelTableIteratorOperation.java | 13 +++++++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index c0e163188611..d3956a4c1790 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -222,7 +222,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, return true; } - private static synchronized boolean checkAndCallFlushToDB(Map localContainerKeyMap, + private static boolean checkAndCallFlushToDB(Map localContainerKeyMap, long containerKeyFlushToDBMaxThreshold, ReconContainerMetadataManager reconContainerMetadataManager) { if (localContainerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java index d85af159d5e6..ea1072543162 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -213,8 +213,17 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, break; } } - } catch (IOException | ExecutionException | InterruptedException e) { - throw new RuntimeException(e); + } catch (IOException e) { + LOG.error("IO error during parallel iteration on table {}", taskName, e); + throw new RuntimeException("IO error during iteration", e); + } catch (InterruptedException e) { + LOG.warn("Parallel iteration interrupted for task {}", taskName, e); + Thread.currentThread().interrupt(); + throw new RuntimeException("Iteration interrupted", e); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + LOG.error("Task execution failed for {}: {}", taskName, cause.getMessage(), cause); + throw new RuntimeException("Task execution failed", cause); } })); } From c02718e447e45d1cd071400e6a8ffa15fc17b016 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 14 Nov 2025 11:16:40 +0530 Subject: [PATCH 12/49] Defined proper caller based locking for writeConuntsToDB --- .../recon/tasks/FileSizeCountTaskHelper.java | 54 +++++++++++++------ 1 file changed, 39 insertions(+), 15 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index aad299842063..8a507f4ddffb 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -48,11 +48,16 @@ public abstract class FileSizeCountTaskHelper { private static final Object TRUNCATE_LOCK = new Object(); /** - * Static write lock for cross-task synchronization during DB writes. - * Ensures read-modify-write operations are atomic when FSO and OBS tasks - * flush concurrently to the same FileSizeCountKey bins. + * GLOBAL lock for cross-task synchronization during DB writes. + * + * Scope: Shared across ALL tasks (FSO, OBS, Legacy) + * Protects: RocksDB read-modify-write operations + * Purpose: Ensures atomic updates when multiple tasks flush concurrently to the same bins + * + * IMPORTANT: Callers MUST acquire this lock before calling writeCountsToDB(). + * This lock should NOT be acquired inside writeCountsToDB() to avoid nested locking. */ - private static final ReentrantReadWriteLock FILE_COUNT_WRITE_LOCK = + public static final ReentrantReadWriteLock FILE_COUNT_WRITE_LOCK = new ReentrantReadWriteLock(); /** @@ -137,7 +142,13 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag long iterationEndTime = Time.monotonicNow(); long writeStartTime = Time.monotonicNow(); - writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + // Acquire GLOBAL lock (cross-task) before writing to DB + FILE_COUNT_WRITE_LOCK.writeLock().lock(); + try { + writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + } finally { + FILE_COUNT_WRITE_LOCK.writeLock().unlock(); + } long writeEndTime = Time.monotonicNow(); long overallEndTime = Time.monotonicNow(); @@ -168,7 +179,8 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); long startTime = Time.monotonicNow(); - // Use fair lock to prevent write lock starvation when flushing + // LOCAL lock (task-specific) - coordinates worker threads within this task only + // Protects: in-memory fileSizeCountMap from concurrent access by workers in THIS task ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); // Flag to coordinate flush attempts - prevents all threads from queuing for write lock AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); @@ -187,21 +199,27 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, if (fileSizeCountMap.size() >= FLUSH_THRESHOLD && isFlushingInProgress.compareAndSet(false, true)) { try { + // Step 1: Acquire LOCAL lock (task-specific) to stop worker threads in THIS task 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); - fileSizeCountMap.clear(); + // Step 2: Acquire GLOBAL lock (cross-task) to protect DB from concurrent FSO/OBS/Legacy writes + FILE_COUNT_WRITE_LOCK.writeLock().lock(); + try { + writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + fileSizeCountMap.clear(); + } finally { + FILE_COUNT_WRITE_LOCK.writeLock().unlock(); // Release GLOBAL lock + } } } finally { isFlushingInProgress.set(false); // Reset flag after flush completes + lock.writeLock().unlock(); // Release LOCAL lock } } catch (Exception e) { throw new RuntimeException("Failed to flush file size counts", e); - } finally { - lock.writeLock().unlock(); } } return null; @@ -283,7 +301,13 @@ public static ReconOmTask.TaskResult processEvents(OMUpdateEventBatch events, } } - writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + // Acquire GLOBAL lock (cross-task) before writing to DB + FILE_COUNT_WRITE_LOCK.writeLock().lock(); + try { + writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + } finally { + FILE_COUNT_WRITE_LOCK.writeLock().unlock(); + } LOG.debug("{} successfully processed using RocksDB in {} milliseconds", taskName, (Time.monotonicNow() - startTime)); @@ -292,6 +316,9 @@ public static ReconOmTask.TaskResult processEvents(OMUpdateEventBatch events, /** * Writes the accumulated file size counts to RocksDB using ReconFileMetadataManager. + * + * IMPORTANT: Caller MUST hold FILE_COUNT_WRITE_LOCK.writeLock() before calling this method. + * This ensures atomic read-modify-write operations across concurrent FSO/OBS task flushes. */ public static void writeCountsToDB(Map fileSizeCountMap, ReconFileMetadataManager reconFileMetadataManager) { @@ -299,8 +326,7 @@ public static void writeCountsToDB(Map fileSizeCountMap, return; } - // Global write lock ensures atomic read-modify-write across concurrent FSO/OBS flushes - FILE_COUNT_WRITE_LOCK.writeLock().lock(); + // Note: Caller must already hold FILE_COUNT_WRITE_LOCK.writeLock() try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { for (Map.Entry entry : fileSizeCountMap.entrySet()) { FileSizeCountKey key = entry.getKey(); @@ -322,8 +348,6 @@ public static void writeCountsToDB(Map fileSizeCountMap, } catch (Exception e) { LOG.error("Error writing file size counts to RocksDB", e); throw new RuntimeException("Failed to write to RocksDB", e); - } finally { - FILE_COUNT_WRITE_LOCK.writeLock().unlock(); } } From 648e94c1e30a7ddf88b704972ed33af490c9b508 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 17 Nov 2025 23:08:14 +0530 Subject: [PATCH 13/49] Some more improvements --- .../recon/tasks/ContainerKeyMapperHelper.java | 40 ++++++++----------- .../tasks/ContainerKeyMapperTaskFSO.java | 2 +- .../recon/tasks/FileSizeCountTaskHelper.java | 2 +- .../recon/tasks/ReconTaskControllerImpl.java | 6 ++- 4 files changed, 24 insertions(+), 26 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index d3956a4c1790..72d049d76ca2 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -136,9 +136,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, int maxWorkers, int maxKeysInMemory) { AtomicLong omKeyCount = new AtomicLong(0); - // Local map: per-task ContainerKeyPrefix mappings (cleared on flush) Map localContainerKeyMap = new ConcurrentHashMap<>(); - // Shared map: cross-task container counts (FSO + OBS accumulate here) try { LOG.info("{}: Starting parallel reprocess with {} iterators, {} workers, max {} keys in memory for bucket layout {}", @@ -199,12 +197,20 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } - // Final flush and commit: local map cleared, shared map kept for other tasks + // Capture total container count BEFORE final flush clears the map + long totalContainers = SHARED_CONTAINER_KEY_COUNT_MAP.size(); + + // Final flush and commit if (!flushAndCommitContainerKeyInfoToDB(localContainerKeyMap, SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager)) { LOG.error("Failed to flush Container Key data to DB for {}", taskName); return false; } + // Write total container count once at the end (after all processing) + if (totalContainers > 0) { + reconContainerMetadataManager.incrementContainerCountBy(totalContainers); + } + Instant end = Instant.now(); long durationMillis = Duration.between(start, end).toMillis(); double durationSeconds = (double) durationMillis / 1000.0; @@ -212,9 +218,9 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, double throughput = keysProcessed / Math.max(durationSeconds, 0.001); LOG.info("{}: Parallel reprocess completed. Processed {} keys in {} ms ({} sec) - " + - "Throughput: {} keys/sec - Containers (shared): {}, Container-Key mappings (local): {}", + "Throughput: {} keys/sec - Containers: {}, Container-Key mappings: {}", taskName, keysProcessed, durationMillis, String.format("%.2f", durationSeconds), - String.format("%.2f", throughput), SHARED_CONTAINER_KEY_COUNT_MAP.size(), localContainerKeyMap.size()); + String.format("%.2f", throughput), totalContainers, localContainerKeyMap.size()); } catch (Exception ex) { LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ex); return false; @@ -495,36 +501,24 @@ public static void handleKeyReprocess(String key, ReconContainerMetadataManager reconContainerMetadataManager) throws IOException { - AtomicLong containerCountToIncrement = new AtomicLong(0); - for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyInfo.getKeyLocationVersions()) { long keyVersion = omKeyLocationInfoGroup.getVersion(); for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfoGroup.getLocationList()) { long containerId = omKeyLocationInfo.getContainerID(); ContainerKeyPrefix containerKeyPrefix = ContainerKeyPrefix.get(containerId, key, keyVersion); - if (reconContainerMetadataManager.getCountForContainerKeyPrefix(containerKeyPrefix) == 0 - && !localContainerKeyMap.containsKey(containerKeyPrefix)) { + // During reprocess, tables are empty so skip DB lookup - just check in-memory map + if (!localContainerKeyMap.containsKey(containerKeyPrefix)) { // Save on writes. No need to save same container-key prefix mapping again. localContainerKeyMap.put(containerKeyPrefix, 1); - // Thread-safe increment using shared AtomicLong map (cross-task safe: FSO + OBS) - AtomicLong count = sharedContainerKeyCountMap.computeIfAbsent(containerId, - k -> new AtomicLong(0)); - - long newCount = count.incrementAndGet(); - - // Check if this is the first key for this container (across all tasks) - if (newCount == 1) { - containerCountToIncrement.incrementAndGet(); - } + // Thread-safe increment using computeIfAbsent (cross-task safe: FSO + OBS) + sharedContainerKeyCountMap.computeIfAbsent(containerId, k -> new AtomicLong(0)) + .incrementAndGet(); } } } - - if (containerCountToIncrement.get() > 0) { - reconContainerMetadataManager.incrementContainerCountBy(containerCountToIncrement.get()); - } + // Container count will be written once at the end of reprocess, not here (Derby optimization) } public static boolean flushAndCommitContainerKeyInfoToDB( diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java index 9b3374248bd3..59f5bdd1efba 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java @@ -66,7 +66,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); boolean result = ContainerKeyMapperHelper.reprocess( omMetadataManager, reconContainerMetadataManager, - BucketLayout.FILE_SYSTEM_OPTIMIZED, getTaskName(), containerKeyFlushToDBMaxThreshold, + BucketLayout.FILE_SYSTEM_OPTIMIZED, getTaskName(), 200000, maxIterators, maxWorkers, maxKeysInMemory); return buildTaskResult(result); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index 8a507f4ddffb..0b865affe098 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -184,7 +184,7 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); // Flag to coordinate flush attempts - prevents all threads from queuing for write lock AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); - final int FLUSH_THRESHOLD = 100000; + final int FLUSH_THRESHOLD = 200000; // Use parallel table iteration Function, Void> kvOperation = kv -> { diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java index f9b3d40d1180..4b98ddd80f09 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java @@ -216,6 +216,7 @@ public synchronized boolean reInitializeTasks(ReconOMMetadataManager omMetadataM }); AtomicBoolean isRunSuccessful = new AtomicBoolean(true); + LOG.info("Submitting {} tasks for parallel reprocessing", tasks.size()); try { CompletableFuture.allOf(tasks.stream() .map(task -> { @@ -223,8 +224,11 @@ public synchronized boolean reInitializeTasks(ReconOMMetadataManager omMetadataM long reprocessStartTime = Time.monotonicNow(); return CompletableFuture.supplyAsync(() -> { + LOG.info("Task {} started execution on thread {}", + task.getTaskName(), Thread.currentThread().getName()); try { ReconOmTask.TaskResult result = task.call(); + LOG.info("Task {} completed execution", task.getTaskName()); return result; } catch (Exception e) { // Track reprocess failure per task @@ -347,7 +351,7 @@ public synchronized void start() { // Clean up any pre-existing checkpoint directories from previous runs cleanupPreExistingCheckpoints(); - executorService = Executors.newFixedThreadPool(threadCount, + executorService = Executors.newFixedThreadPool(8, new ThreadFactoryBuilder().setNameFormat("ReconTaskThread-%d") .build()); From 0c0c09a679daca227c35e954f03e254d34db5d13 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 24 Nov 2025 11:00:29 +0530 Subject: [PATCH 14/49] Added changes to the parallel Iterator class --- .../util/ParallelTableIteratorOperation.java | 61 +++++++++++++++---- 1 file changed, 50 insertions(+), 11 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java index ea1072543162..bee48af7ade6 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -28,11 +28,11 @@ 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.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -53,8 +53,12 @@ public class ParallelTableIteratorOperation, V> implements Closeable { private final Table table; private final Codec keyCodec; - private final ExecutorService iteratorExecutor; - private final ExecutorService valueExecutors; + + // Thread Pools + private final ExecutorService iteratorExecutor; // 5 + private final ExecutorService valueExecutors; // 20 + + private final int maxNumberOfVals; private final OMMetadataManager metadataManager; private final int maxIteratorTasks; @@ -68,14 +72,19 @@ public ParallelTableIteratorOperation(OMMetadataManager metadataManager, Table(iteratorCount * 2), - new ThreadPoolExecutor.CallerRunsPolicy()); + new LinkedBlockingQueue<>()); + + // Create team of 20 worker threads with UNLIMITED queue this.valueExecutors = new ThreadPoolExecutor(workerCount, workerCount, 1, TimeUnit.MINUTES, - new ArrayBlockingQueue<>(workerCount * 2), - new ThreadPoolExecutor.CallerRunsPolicy()); + new LinkedBlockingQueue<>()); + + // Calculate batch size per worker (e.g., 2000 / 20 = 100 keys per batch per worker) this.maxNumberOfVals = Math.max(10, maxNumberOfValsInMemory / (workerCount)); this.logCountThreshold = logThreshold; } @@ -131,6 +140,7 @@ private void waitForQueueSize(Queue> futures, int expectedSize) } } + // Main parallelization logic public void performTaskOnTableVals(String taskName, K startKey, K endKey, Function, Void> keyOperation) throws IOException, ExecutionException, InterruptedException { List bounds = getBounds(startKey, endKey); @@ -151,16 +161,32 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, } return; } - + + // ===== PARALLEL PROCESSING SETUP ===== + + // Queue to track iterator threads (5 threads creating work) Queue> iterFutures = new LinkedList<>(); + + // Queue to track worker threads (20 threads doing work) Queue> workerFutures = new ConcurrentLinkedQueue<>(); + AtomicLong keyCounter = new AtomicLong(); AtomicLong prevLogCounter = new AtomicLong(); + + // ===== STEP 2: START ITERATOR THREADS ===== + // For each segment boundary, create an iterator thread + // Example: If bounds = [0, 5M, 10M, 15M, 20M], this loop runs 4 times: + // idx=1: beg=0, end=5M + // idx=2: beg=5M, end=10M + // idx=3: beg=10M, end=15M + // idx=4: beg=15M, end=20M 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); + + // ===== STEP 3: SUBMIT ITERATOR TASK ===== iterFutures.add(iteratorExecutor.submit(() -> { try (TableIterator> iter = table.iterator()) { iter.seek(beg); @@ -187,12 +213,19 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, reachedEnd = true; break; } + + // If batch is full (2000 keys), stop collecting if (keyValues.size() >= maxNumberOfVals) { break; } } + + // ===== STEP 5: HAND BATCH TO WORKER THREAD ===== if (!keyValues.isEmpty()) { - waitForQueueSize(workerFutures, maxWorkerTasks - 10); + // WAIT if worker queue is too full (max 39 pending tasks) + waitForQueueSize(workerFutures, maxWorkerTasks - 1); + + // Submit batch to worker thread pool workerFutures.add(valueExecutors.submit(() -> { for (Table.KeyValue kv : keyValues) { keyOperation.apply(kv); @@ -207,8 +240,10 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, } } } + // Worker task done! Future is now complete. })); } + // If we reached the end of our segment, stop reading if (reachedEnd) { break; } @@ -227,7 +262,11 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, } })); } + + // ===== STEP 7: WAIT FOR EVERYONE TO FINISH ===== + // Wait for all 5 iterator threads to finish reading waitForQueueSize(iterFutures, 0); + // Wait for all 20 worker threads to finish processing waitForQueueSize(workerFutures, 0); // Log final stats From f7e0cf3ab5b1603ede447801b399a86b8950a24e Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 24 Nov 2025 12:01:03 +0530 Subject: [PATCH 15/49] Consolidated all the locks in a single lock for simplicity --- .../hadoop/ozone/recon/ReconConstants.java | 9 +- .../recon/tasks/ContainerKeyMapperHelper.java | 101 +++++++++--------- 2 files changed, 53 insertions(+), 57 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java index 890253839ec7..216610bde673 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java @@ -93,9 +93,9 @@ public final class ReconConstants { // For file-size count reprocessing: ensure only one task truncates the table. public static final AtomicBoolean FILE_SIZE_COUNT_TABLE_TRUNCATED = new AtomicBoolean(false); - public static final AtomicBoolean CONTAINER_KEY_TABLES_TRUNCATED = new AtomicBoolean(false); - - public static final AtomicBoolean CONTAINER_KEY_COUNT_MAP_INITIALIZED = new AtomicBoolean(false); + // For container key mapper reprocessing: ensure only one task performs initialization + // (truncates tables + clears shared map) + public static final AtomicBoolean CONTAINER_KEY_MAPPER_INITIALIZED = new AtomicBoolean(false); private ReconConstants() { // Never Constructed @@ -107,7 +107,6 @@ private ReconConstants() { */ public static void resetTableTruncatedFlags() { FILE_SIZE_COUNT_TABLE_TRUNCATED.set(false); - CONTAINER_KEY_TABLES_TRUNCATED.set(false); - CONTAINER_KEY_COUNT_MAP_INITIALIZED.set(false); + CONTAINER_KEY_MAPPER_INITIALIZED.set(false); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 72d049d76ca2..1783463ec423 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; @@ -59,8 +60,8 @@ public abstract class ContainerKeyMapperHelper { private static final Logger LOG = LoggerFactory.getLogger(ContainerKeyMapperHelper.class); - // Static lock to guard table truncation. - private static final Object TRUNCATE_LOCK = new Object(); + // Single lock to guard all initialization operations (table truncation + map clearing) + private static final Object INITIALIZATION_LOCK = new Object(); /** * SHARED across all tasks (FSO + OBS) for cross-task synchronization. @@ -71,62 +72,48 @@ public abstract class ContainerKeyMapperHelper { */ private static final Map SHARED_CONTAINER_KEY_COUNT_MAP = new ConcurrentHashMap<>(); - // Lock to guard shared map initialization/clearing - private static final Object SHARED_MAP_LOCK = new Object(); - /** - * Ensures that the container key tables are truncated only once before reprocessing. - * Uses an AtomicBoolean to track if truncation has already been performed. - * - * @param reconContainerMetadataManager The metadata manager instance responsible for DB operations. + * Performs one-time initialization for Container Key Mapper tasks. + * This includes: + * 1. Truncating container key tables in DB + * 2. Clearing the shared container count map + * + * This method is called by both FSO and OBS tasks at the start of reprocess. + * Only the first task to call this will perform initialization. + * + * @param reconContainerMetadataManager The metadata manager for DB operations + * @param taskName Name of the task calling this method (for logging) + * @throws RuntimeException if initialization fails */ - public static void truncateTablesIfNeeded(ReconContainerMetadataManager reconContainerMetadataManager, - String taskName) { - synchronized (TRUNCATE_LOCK) { - if (ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.compareAndSet(false, true)) { + private static void initializeContainerKeyMapperIfNeeded( + ReconContainerMetadataManager reconContainerMetadataManager, + String taskName) { + + synchronized (INITIALIZATION_LOCK) { + // Check if already initialized by another task + if (ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.compareAndSet(false, true)) { try { - // Perform table truncation + // Step 1: Truncate tables + LOG.info("{}: Truncating container key tables for reprocess", taskName); reconContainerMetadataManager.reinitWithNewContainerDataFromOm(Collections.emptyMap()); - LOG.debug("Successfully truncated container key tables."); + LOG.info("{}: Successfully truncated container key tables", taskName); + + // Step 2: Clear shared map + SHARED_CONTAINER_KEY_COUNT_MAP.clear(); + LOG.info("{}: Initialized shared container count map for cross-task synchronization", taskName); + } catch (Exception e) { - // Reset the flag so truncation can be retried - ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.set(false); - LOG.error("Error while truncating container key tables for task {}. Resetting flag.", taskName, e); - throw new RuntimeException("Table truncation failed", e); + // CRITICAL: Reset flag so another task can retry + ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); + LOG.error("{}: Container Key Mapper initialization failed. Resetting flag for retry.", taskName, e); + throw new RuntimeException("Container Key Mapper initialization failed", e); } } else { - LOG.debug("Container key tables already truncated by another task."); - } - } - } - - /** - * Ensures the shared container count map is cleared once per reprocess cycle. - * This must be called by the first task that starts reprocessing to prevent - * cross-task data corruption where FSO and OBS tasks overwrite each other's counts. - */ - private static void initializeSharedContainerCountMapIfNeeded(String taskName) { - synchronized (SHARED_MAP_LOCK) { - if (ReconConstants.CONTAINER_KEY_COUNT_MAP_INITIALIZED.compareAndSet(false, true)) { - SHARED_CONTAINER_KEY_COUNT_MAP.clear(); - LOG.info("{}: Initialized shared container count map for cross-task synchronization", taskName); - } else { - LOG.debug("{}: Shared container count map already initialized by another task", taskName); + LOG.debug("{}: Container Key Mapper already initialized by another task", taskName); } } } - /** - * Clears the shared container count map and resets its initialization flag. - * This method should be called by tests to ensure clean state between test runs. - */ - public static void clearSharedContainerCountMap() { - synchronized (SHARED_MAP_LOCK) { - SHARED_CONTAINER_KEY_COUNT_MAP.clear(); - ReconConstants.CONTAINER_KEY_COUNT_MAP_INITIALIZED.set(false); - } - } - public static boolean reprocess(OMMetadataManager omMetadataManager, ReconContainerMetadataManager reconContainerMetadataManager, BucketLayout bucketLayout, @@ -143,11 +130,8 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, taskName, maxIterators, maxWorkers, maxKeysInMemory, bucketLayout); Instant start = Instant.now(); - // Ensure the tables are truncated only once - truncateTablesIfNeeded(reconContainerMetadataManager, taskName); - - // Initialize shared container count map once per cycle - initializeSharedContainerCountMapIfNeeded(taskName); + // Perform one-time initialization (truncate tables + clear shared map) + initializeContainerKeyMapperIfNeeded(reconContainerMetadataManager, taskName); // Get the appropriate table based on BucketLayout Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); @@ -540,4 +524,17 @@ public static boolean flushAndCommitContainerKeyInfoToDB( return true; } + /** + * Clears the shared container count map and resets initialization flag. + * This method should be called by tests to ensure clean state between test runs. + */ + @VisibleForTesting + public static void clearSharedContainerCountMap() { + synchronized (INITIALIZATION_LOCK) { + SHARED_CONTAINER_KEY_COUNT_MAP.clear(); + ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); + LOG.debug("Cleared shared container count map and reset initialization flag for tests"); + } + } + } From f4ddf9e54de1affdca148123842da495eec257ff Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 24 Nov 2025 13:27:07 +0530 Subject: [PATCH 16/49] Fixed memory leak by clearing the shared map --- .../recon/tasks/ContainerKeyMapperHelper.java | 32 +++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 1783463ec423..cbbc87b8d501 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -31,6 +31,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; @@ -63,6 +64,13 @@ public abstract class ContainerKeyMapperHelper { // Single lock to guard all initialization operations (table truncation + map clearing) private static final Object INITIALIZATION_LOCK = new Object(); + /** + * Reference counter to track how many tasks are actively using the shared map. + * Initialized to 2 (FSO + OBS tasks) during initialization. + * Each task decrements on completion. Last task (count reaches 0) clears the shared map. + */ + private static final AtomicInteger ACTIVE_TASK_COUNT = new AtomicInteger(0); + /** * SHARED across all tasks (FSO + OBS) for cross-task synchronization. * Maps: ContainerId -> AtomicLong (key count in that container) @@ -102,6 +110,10 @@ private static void initializeContainerKeyMapperIfNeeded( SHARED_CONTAINER_KEY_COUNT_MAP.clear(); LOG.info("{}: Initialized shared container count map for cross-task synchronization", taskName); + // Step 3: Initialize reference counter (2 tasks: FSO + OBS) + ACTIVE_TASK_COUNT.set(2); + LOG.info("{}: Initialized active task counter to 2", taskName); + } catch (Exception e) { // CRITICAL: Reset flag so another task can retry ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); @@ -136,7 +148,6 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Get the appropriate table based on BucketLayout Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); - // Use fair lock to prevent write lock starvation when flushing ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); // Flag to coordinate flush attempts - prevents all threads from queuing for write lock AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); @@ -516,6 +527,22 @@ public static boolean flushAndCommitContainerKeyInfoToDB( // Only clear localContainerKeyMap (per-task), keep sharedContainerKeyCountMap for other tasks localContainerKeyMap.clear(); + + // If this is a final flush (sharedContainerKeyCountMap is not empty), + // decrement reference counter and cleanup if last task + if (!sharedContainerKeyCountMap.isEmpty()) { + int remainingTasks = ACTIVE_TASK_COUNT.decrementAndGet(); + LOG.info("Task completed. Remaining active tasks: {}", remainingTasks); + + if (remainingTasks == 0) { + // Last task finished - clean up shared resources + synchronized (INITIALIZATION_LOCK) { + SHARED_CONTAINER_KEY_COUNT_MAP.clear(); + ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); + LOG.info("Last task completed. Cleared shared map and reset initialization flag."); + } + } + } } catch (IOException e) { LOG.error("Unable to write Container Key and Container Key Count data in Recon DB.", e); @@ -533,7 +560,8 @@ public static void clearSharedContainerCountMap() { synchronized (INITIALIZATION_LOCK) { SHARED_CONTAINER_KEY_COUNT_MAP.clear(); ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); - LOG.debug("Cleared shared container count map and reset initialization flag for tests"); + ACTIVE_TASK_COUNT.set(0); + LOG.debug("Cleared shared container count map, reset initialization flag, and reset task counter for tests"); } } From 28719bf78a7641c2137548a91d6d76871fc0eb97 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 24 Nov 2025 13:47:32 +0530 Subject: [PATCH 17/49] Refactored the code --- .../recon/tasks/ContainerKeyMapperHelper.java | 32 ++++++++----------- 1 file changed, 14 insertions(+), 18 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index cbbc87b8d501..6d06fa10e11a 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -112,8 +112,7 @@ private static void initializeContainerKeyMapperIfNeeded( // Step 3: Initialize reference counter (2 tasks: FSO + OBS) ACTIVE_TASK_COUNT.set(2); - LOG.info("{}: Initialized active task counter to 2", taskName); - + } catch (Exception e) { // CRITICAL: Reset flag so another task can retry ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); @@ -206,6 +205,19 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, reconContainerMetadataManager.incrementContainerCountBy(totalContainers); } + // Decrement active task counter and cleanup if this is the last task + int remainingTasks = ACTIVE_TASK_COUNT.decrementAndGet(); + LOG.info("{}: Task completed. Remaining active tasks: {}", taskName, remainingTasks); + + if (remainingTasks == 0) { + // Last task finished - clean up shared resources + synchronized (INITIALIZATION_LOCK) { + SHARED_CONTAINER_KEY_COUNT_MAP.clear(); + ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); + LOG.debug("{}: Last task completed. Cleared shared map and reset initialization flag.", taskName); + } + } + Instant end = Instant.now(); long durationMillis = Duration.between(start, end).toMillis(); double durationSeconds = (double) durationMillis / 1000.0; @@ -527,22 +539,6 @@ public static boolean flushAndCommitContainerKeyInfoToDB( // Only clear localContainerKeyMap (per-task), keep sharedContainerKeyCountMap for other tasks localContainerKeyMap.clear(); - - // If this is a final flush (sharedContainerKeyCountMap is not empty), - // decrement reference counter and cleanup if last task - if (!sharedContainerKeyCountMap.isEmpty()) { - int remainingTasks = ACTIVE_TASK_COUNT.decrementAndGet(); - LOG.info("Task completed. Remaining active tasks: {}", remainingTasks); - - if (remainingTasks == 0) { - // Last task finished - clean up shared resources - synchronized (INITIALIZATION_LOCK) { - SHARED_CONTAINER_KEY_COUNT_MAP.clear(); - ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); - LOG.info("Last task completed. Cleared shared map and reset initialization flag."); - } - } - } } catch (IOException e) { LOG.error("Unable to write Container Key and Container Key Count data in Recon DB.", e); From d3361f6e0dfc83f8997d3e541cdfdf130c05b2fc Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 24 Nov 2025 16:21:53 +0530 Subject: [PATCH 18/49] Made the lock for the file size count task local rather than global --- .../recon/tasks/FileSizeCountTaskHelper.java | 60 +++++-------------- 1 file changed, 15 insertions(+), 45 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index 0b865affe098..24e217d02170 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -47,19 +47,6 @@ public abstract class FileSizeCountTaskHelper { // Static lock object for table truncation synchronization private static final Object TRUNCATE_LOCK = new Object(); - /** - * GLOBAL lock for cross-task synchronization during DB writes. - * - * Scope: Shared across ALL tasks (FSO, OBS, Legacy) - * Protects: RocksDB read-modify-write operations - * Purpose: Ensures atomic updates when multiple tasks flush concurrently to the same bins - * - * IMPORTANT: Callers MUST acquire this lock before calling writeCountsToDB(). - * This lock should NOT be acquired inside writeCountsToDB() to avoid nested locking. - */ - public static final ReentrantReadWriteLock FILE_COUNT_WRITE_LOCK = - new ReentrantReadWriteLock(); - /** * Increments the count for a given key on a PUT event. */ @@ -142,13 +129,8 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag long iterationEndTime = Time.monotonicNow(); long writeStartTime = Time.monotonicNow(); - // Acquire GLOBAL lock (cross-task) before writing to DB - FILE_COUNT_WRITE_LOCK.writeLock().lock(); - try { - writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); - } finally { - FILE_COUNT_WRITE_LOCK.writeLock().unlock(); - } + // Write remaining counts to DB (no global lock needed - FSO and OBS are mutually exclusive) + writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); long writeEndTime = Time.monotonicNow(); long overallEndTime = Time.monotonicNow(); @@ -178,9 +160,8 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, taskName, maxIterators, maxWorkers, bucketLayout); Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); long startTime = Time.monotonicNow(); - + // LOCAL lock (task-specific) - coordinates worker threads within this task only - // Protects: in-memory fileSizeCountMap from concurrent access by workers in THIS task ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); // Flag to coordinate flush attempts - prevents all threads from queuing for write lock AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); @@ -197,26 +178,21 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, // Only one thread should attempt flush to avoid blocking all workers if (fileSizeCountMap.size() >= FLUSH_THRESHOLD && + // If current value is false: Set to true and return true + // If current value is true: Do nothing and return false isFlushingInProgress.compareAndSet(false, true)) { try { - // Step 1: Acquire LOCAL lock (task-specific) to stop worker threads in THIS task lock.writeLock().lock(); try { - // Double-check after acquiring write lock + // Double-check size after acquiring write lock if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { LOG.debug("Flushing {} accumulated counts to RocksDB for {}", fileSizeCountMap.size(), taskName); - // Step 2: Acquire GLOBAL lock (cross-task) to protect DB from concurrent FSO/OBS/Legacy writes - FILE_COUNT_WRITE_LOCK.writeLock().lock(); - try { - writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); - fileSizeCountMap.clear(); - } finally { - FILE_COUNT_WRITE_LOCK.writeLock().unlock(); // Release GLOBAL lock - } + writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + fileSizeCountMap.clear(); } } finally { - isFlushingInProgress.set(false); // Reset flag after flush completes - lock.writeLock().unlock(); // Release LOCAL lock + isFlushingInProgress.set(false); + lock.writeLock().unlock(); } } catch (Exception e) { throw new RuntimeException("Failed to flush file size counts", e); @@ -301,13 +277,8 @@ public static ReconOmTask.TaskResult processEvents(OMUpdateEventBatch events, } } - // Acquire GLOBAL lock (cross-task) before writing to DB - FILE_COUNT_WRITE_LOCK.writeLock().lock(); - try { - writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); - } finally { - FILE_COUNT_WRITE_LOCK.writeLock().unlock(); - } + // Write remaining counts to DB (no lock needed for incremental processing) + writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); LOG.debug("{} successfully processed using RocksDB in {} milliseconds", taskName, (Time.monotonicNow() - startTime)); @@ -317,8 +288,8 @@ public static ReconOmTask.TaskResult processEvents(OMUpdateEventBatch events, /** * Writes the accumulated file size counts to RocksDB using ReconFileMetadataManager. * - * IMPORTANT: Caller MUST hold FILE_COUNT_WRITE_LOCK.writeLock() before calling this method. - * This ensures atomic read-modify-write operations across concurrent FSO/OBS task flushes. + * Thread Safety: FSO and OBS tasks write to different bucket keys (mutually exclusive), + * so no global lock is needed. RocksDB handles concurrent writes to different keys safely. */ public static void writeCountsToDB(Map fileSizeCountMap, ReconFileMetadataManager reconFileMetadataManager) { @@ -326,13 +297,12 @@ public static void writeCountsToDB(Map fileSizeCountMap, return; } - // Note: Caller must already hold FILE_COUNT_WRITE_LOCK.writeLock() try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { for (Map.Entry entry : fileSizeCountMap.entrySet()) { FileSizeCountKey key = entry.getKey(); Long deltaCount = entry.getValue(); - // Always do read-modify-write to handle concurrent task flushes correctly + // Read-modify-write: Read current count, add delta, write back Long existingCount = reconFileMetadataManager.getFileSizeCount(key); Long newCount = (existingCount != null ? existingCount : 0L) + deltaCount; From ff3c6e98f493935c2c9b230de3689496bc19f170 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 24 Nov 2025 17:41:35 +0530 Subject: [PATCH 19/49] Remove un-necessary code from OzoneManagerServiceProviderImpl --- .../impl/OzoneManagerServiceProviderImpl.java | 304 +++++++++--------- 1 file changed, 148 insertions(+), 156 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 0b5a9a9e92bb..874049bfa89e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -114,7 +114,7 @@ @Singleton public class OzoneManagerServiceProviderImpl implements OzoneManagerServiceProvider { - private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); + private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; @@ -346,17 +346,17 @@ private void startSyncDataFromOM(long initialDelay) { TimeUnit.MILLISECONDS); LOG.debug("Started the OM DB sync scheduler."); scheduler.scheduleWithFixedDelay(() -> { - try { - LOG.info("Last known sequence number before sync: {}", getCurrentOMDBSequenceNumber()); - boolean isSuccess = syncDataFromOM(); - if (!isSuccess) { - LOG.debug("OM DB sync is already running, or encountered an error while trying to sync data."); - } - LOG.info("Sequence number after sync: {}", getCurrentOMDBSequenceNumber()); - } catch (Throwable t) { - LOG.error("Unexpected exception while syncing data from OM.", t); - } - }, + try { + LOG.info("Last known sequence number before sync: {}", getCurrentOMDBSequenceNumber()); + boolean isSuccess = syncDataFromOM(); + if (!isSuccess) { + LOG.debug("OM DB sync is already running, or encountered an error while trying to sync data."); + } + LOG.info("Sequence number after sync: {}", getCurrentOMDBSequenceNumber()); + } catch (Throwable t) { + LOG.error("Unexpected exception while syncing data from OM.", t); + } + }, initialDelay, interval, TimeUnit.MILLISECONDS); @@ -702,153 +702,145 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = true; - + boolean fullSnapshot = false; - if (reInitializeTasksCalled.compareAndSet(false, true)) { - LOG.info("Calling reprocess on Recon tasks."); - reconTaskController.reInitializeTasks(omMetadataManager, null); + if (currentSequenceNumber <= 0) { + fullSnapshot = true; } else { - LOG.info("reInitializeTasks already called once; skipping."); + // Get updates from OM and apply to local Recon OM DB and update task status in table + deltaReconTaskStatusUpdater.recordRunStart(); + int loopCount = 0; + long fromSequenceNumber = currentSequenceNumber; + long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; + /** + * This loop will continue to fetch and apply OM DB updates and with every + * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. + * It continues to fetch from OM till the lag, between OM DB WAL sequence number + * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. + * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. + */ + while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { + try (OMDBUpdatesHandler omdbUpdatesHandler = + new OMDBUpdatesHandler(omMetadataManager)) { + + // If interrupt was previously signalled, + // we should check for it before starting delta update sync. + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted during delta update."); + } + diffBetweenOMDbAndReconDBSeqNumber = + getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); + deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); + // Keeping last updated sequence number for both full and delta tasks to be same + // because sequence number of DB denotes and points to same OM DB copy of Recon, + // even though two different tasks are updating the DB at different conditions, but + // it tells the sync state with actual OM DB for the same Recon OM DB copy. + deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + deltaReconTaskStatusUpdater.recordRunCompletion(); + fullSnapshotReconTaskUpdater.updateDetails(); + // Update the current OM metadata manager in task controller + reconTaskController.updateOMMetadataManager(omMetadataManager); + + // Pass on DB update events to tasks that are listening. + reconTaskController.consumeOMEvents(new OMUpdateEventBatch( + omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); + + // Check if task reinitialization is needed due to buffer overflow or task failures + boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); + boolean tasksFailed = reconTaskController.hasTasksFailed(); + + if (bufferOverflowed || tasksFailed) { + ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? + ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : + ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; + + LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", + reason); + + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + + // Queue async reinitialization event - checkpoint creation and retry logic is handled internally + ReconTaskController.ReInitializationResult result = + reconTaskController.queueReInitializationEvent(reason); + + //TODO: Create a metric to track this event buffer overflow or task failure event + boolean triggerFullSnapshot = + Optional.ofNullable(result) + .map(r -> { + switch (r) { + case MAX_RETRIES_EXCEEDED: + LOG.warn( + "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + + "fallback"); + return true; + + case RETRY_LATER: + LOG.debug("Reinitialization event queueing will be retried in next iteration"); + return false; + + default: + LOG.info("Reinitialization event successfully queued"); + return false; + } + }) + .orElseGet(() -> { + LOG.error( + "ReInitializationResult is null, something went wrong in queueing reinitialization " + + "event"); + return true; + }); + + if (triggerFullSnapshot) { + fullSnapshot = true; + } + } + currentSequenceNumber = getCurrentOMDBSequenceNumber(); + LOG.debug("Updated current sequence number: {}", currentSequenceNumber); + loopCount++; + } catch (InterruptedException intEx) { + LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); + // We are updating the table even if it didn't run i.e. got interrupted beforehand + // to indicate that a task was supposed to run, but it didn't. + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + Thread.currentThread().interrupt(); + // Since thread is interrupted, we do not fall back to snapshot sync. + // Return with sync failed status. + return false; + } catch (Exception e) { + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", + e.getMessage()); + fullSnapshot = true; + } + if (fullSnapshot) { + break; + } + } + LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, + getCurrentOMDBSequenceNumber() - fromSequenceNumber); } -// if (currentSequenceNumber <= 0) { -// fullSnapshot = true; -// } else { -// // Get updates from OM and apply to local Recon OM DB and update task status in table -// deltaReconTaskStatusUpdater.recordRunStart(); -// int loopCount = 0; -// long fromSequenceNumber = currentSequenceNumber; -// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; -// /** -// * This loop will continue to fetch and apply OM DB updates and with every -// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. -// * It continues to fetch from OM till the lag, between OM DB WAL sequence number -// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. -// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. -// */ -// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { -// try (OMDBUpdatesHandler omdbUpdatesHandler = -// new OMDBUpdatesHandler(omMetadataManager)) { -// -// // If interrupt was previously signalled, -// // we should check for it before starting delta update sync. -// if (Thread.currentThread().isInterrupted()) { -// throw new InterruptedException("Thread interrupted during delta update."); -// } -// diffBetweenOMDbAndReconDBSeqNumber = -// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); -// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); -// // Keeping last updated sequence number for both full and delta tasks to be same -// // because sequence number of DB denotes and points to same OM DB copy of Recon, -// // even though two different tasks are updating the DB at different conditions, but -// // it tells the sync state with actual OM DB for the same Recon OM DB copy. -// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// deltaReconTaskStatusUpdater.recordRunCompletion(); -// fullSnapshotReconTaskUpdater.updateDetails(); -// // Update the current OM metadata manager in task controller -// reconTaskController.updateOMMetadataManager(omMetadataManager); -// -// // Pass on DB update events to tasks that are listening. -// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( -// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); -// -// // Check if task reinitialization is needed due to buffer overflow or task failures -// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); -// boolean tasksFailed = reconTaskController.hasTasksFailed(); -// -// if (bufferOverflowed || tasksFailed) { -// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? -// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : -// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; -// -// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", -// reason); -// -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// -// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally -// ReconTaskController.ReInitializationResult result = -// reconTaskController.queueReInitializationEvent(reason); -// -// //TODO: Create a metric to track this event buffer overflow or task failure event -// boolean triggerFullSnapshot = -// Optional.ofNullable(result) -// .map(r -> { -// switch (r) { -// case MAX_RETRIES_EXCEEDED: -// LOG.warn( -// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + -// "fallback"); -// return true; -// -// case RETRY_LATER: -// LOG.debug("Reinitialization event queueing will be retried in next iteration"); -// return false; -// -// default: -// LOG.info("Reinitialization event successfully queued"); -// return false; -// } -// }) -// .orElseGet(() -> { -// LOG.error( -// "ReInitializationResult is null, something went wrong in queueing reinitialization " + -// "event"); -// return true; -// }); -// -// if (triggerFullSnapshot) { -// fullSnapshot = true; -// } -// } -// currentSequenceNumber = getCurrentOMDBSequenceNumber(); -// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); -// loopCount++; -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); -// // We are updating the table even if it didn't run i.e. got interrupted beforehand -// // to indicate that a task was supposed to run, but it didn't. -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// Thread.currentThread().interrupt(); -// // Since thread is interrupted, we do not fall back to snapshot sync. -// // Return with sync failed status. -// return false; -// } catch (Exception e) { -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", -// e.getMessage()); -// fullSnapshot = true; -// } -// if (fullSnapshot) { -// break; -// } -// } -// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, -// getCurrentOMDBSequenceNumber() - fromSequenceNumber); -// } - -// if (fullSnapshot) { -// try { -// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Snapshot update sync thread was interrupted."); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// Thread.currentThread().interrupt(); -// // Mark sync status as failed. -// return false; -// } catch (Exception e) { -// metrics.incrNumSnapshotRequestsFailed(); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); -// // Update health status in ReconContext -// reconContext.updateHealthStatus(new AtomicBoolean(false)); -// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); -// } -// } + if (fullSnapshot) { + try { + executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); + } catch (InterruptedException intEx) { + LOG.error("OM DB Snapshot update sync thread was interrupted."); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + Thread.currentThread().interrupt(); + // Mark sync status as failed. + return false; + } catch (Exception e) { + metrics.incrNumSnapshotRequestsFailed(); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + LOG.error("Unable to update Recon's metadata with new OM DB. ", e); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); + } + } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); @@ -867,7 +859,7 @@ private void markDeltaTaskStatusAsFailed(ReconTaskStatusUpdater deltaReconTaskSt } private void executeFullSnapshot(ReconTaskStatusUpdater fullSnapshotReconTaskUpdater, - ReconTaskStatusUpdater deltaReconTaskStatusUpdater) throws InterruptedException, IOException { + ReconTaskStatusUpdater deltaReconTaskStatusUpdater) throws InterruptedException, IOException { metrics.incrNumSnapshotRequests(); LOG.info("Obtaining full snapshot from Ozone Manager"); From ce8a0bd17980502b6acbf7e5bd41c61bbaaf31b2 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 26 Nov 2025 01:15:21 +0530 Subject: [PATCH 20/49] Added Parallelization for the OmtableInsightTask --- .../ozone/recon/tasks/OmTableInsightTask.java | 150 ++++++++++-------- 1 file changed, 85 insertions(+), 65 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index d4e9b151f0fd..91ef882949e5 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java @@ -39,8 +39,6 @@ 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.apache.hadoop.ozone.om.helpers.OmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconGlobalStatsManager; @@ -66,7 +64,6 @@ public class OmTableInsightTask implements ReconOmTask { private Map replicatedSizeMap; private final int maxKeysInMemory; private final int maxIterators; - private final int maxWorkers; @Inject public OmTableInsightTask(ReconGlobalStatsManager reconGlobalStatsManager, @@ -85,9 +82,6 @@ public OmTableInsightTask(ReconGlobalStatsManager reconGlobalStatsManager, this.maxIterators = reconOMMetadataManager.getOzoneConfiguration().getInt( ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS, ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS_DEFAULT); - this.maxWorkers = reconOMMetadataManager.getOzoneConfiguration().getInt( - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, - ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); } @Override @@ -113,16 +107,13 @@ public void init() { } /** - * Iterates the rows of each table in the OM snapshot DB and calculates the - * counts and sizes for table data. - *

- * For tables that require data size calculation - * (as returned by getTablesToCalculateSize), both the number of - * records (count) and total data size of the records are calculated. - * For all other tables, only the count of records is calculated. + * Reprocess all OM tables to calculate counts and sizes. + * Handler tables (with size calculation) use sequential iteration. + * Simple tables (count only) use parallel iteration with String keys, + * or sequential for non-String key tables. * - * @param omMetadataManager OM Metadata instance. - * @return Pair + * @param omMetadataManager OM Metadata instance + * @return TaskResult indicating success or failure */ @Override public TaskResult reprocess(OMMetadataManager omMetadataManager) { @@ -131,25 +122,27 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { init(); for (String tableName : tables) { - Table table = omMetadataManager.getTable(tableName); + Table table = omMetadataManager.getTable(tableName); try { if (tableHandlers.containsKey(tableName)) { - // Parallelize handler tables (OPEN_KEY_TABLE, OPEN_FILE_TABLE, DELETED_TABLE) - LOG.info("{}: Processing table {} with parallel iteration ({} iterators, {} workers)", - getTaskName(), tableName, maxIterators, maxWorkers); - - Triple details = getTableSizeAndCountParallel( - omMetadataManager, tableName, tableHandlers.get(tableName)); - - objectCountMap.put(getTableCountKeyFromTable(tableName), details.getLeft()); - unReplicatedSizeMap.put(getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); - replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), details.getRight()); + Table stringTable = (Table) table; + try (TableIterator> iterator + = stringTable.iterator()) { + Triple details = + tableHandlers.get(tableName).getTableSizeAndCount(iterator); + objectCountMap.put(getTableCountKeyFromTable(tableName), + details.getLeft()); + unReplicatedSizeMap.put( + getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); + replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), + details.getRight()); + } } else { - // Keep simple tables sequential due to heterogeneous key types - try (TableIterator> iterator = table.iterator()) { - long count = Iterators.size(iterator); - objectCountMap.put(getTableCountKeyFromTable(tableName), count); + if (usesNonStringKeys(tableName)) { + processTableSequentially(tableName, table); + } else { + processTableInParallel(tableName, table, omMetadataManager); } } } catch (Exception ioEx) { @@ -170,58 +163,85 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { long endTime = Time.monotonicNow(); long durationMs = endTime - startTime; double durationSec = durationMs / 1000.0; - int parallelizedTables = tableHandlers.size(); // 3 handler tables - int sequentialTables = tables.size() - parallelizedTables; + int handlerTables = tableHandlers.size(); // 3 handler tables (with size calculation) + int simpleTables = tables.size() - handlerTables; // Simple count-only tables LOG.info("{}: RocksDB reprocess completed in {} ms ({} sec) - " + - "Total tables: {}, Parallelized (with handlers): {}, Sequential (simple count): {}", + "Total tables: {}, Handler tables (with size): {}, Simple tables (count only): {}", getTaskName(), durationMs, String.format("%.2f", durationSec), - tables.size(), parallelizedTables, sequentialTables); + tables.size(), handlerTables, simpleTables); return buildTaskResult(true); } /** - * Parallel version of getTableSizeAndCount for handler tables. - * Uses ParallelTableIteratorOperation with atomic accumulators. + * Check if table uses non-String keys (e.g., OzoneTokenIdentifier). + * These tables cannot use StringCodec and must be processed sequentially. + */ + private boolean usesNonStringKeys(String tableName) { + return tableName.equals("dTokenTable") || tableName.equals("s3SecretTable"); + } + + /** + * Process table sequentially using raw iterator (no type assumptions). + * Used for tables with non-String keys or as fallback. */ - private Triple getTableSizeAndCountParallel( - OMMetadataManager omMetadataManager, String tableName, OmTableHandler handler) - throws Exception { + private void processTableSequentially(String tableName, Table table) throws IOException { + LOG.info("{}: Processing table {} sequentially (non-String keys)", + getTaskName(), tableName); - AtomicLong count = new AtomicLong(0); - AtomicLong unReplicatedSize = new AtomicLong(0); - AtomicLong replicatedSize = new AtomicLong(0); + Table stringTable = (Table) table; + try (TableIterator> iterator = stringTable.iterator()) { + long count = Iterators.size(iterator); + objectCountMap.put(getTableCountKeyFromTable(tableName), count); + } + } + + /** + * Process table in parallel using multiple iterators and workers. + * Only for tables with String keys. + */ + private void processTableInParallel(String tableName, Table table, + OMMetadataManager omMetadataManager) throws Exception { + int workerCount = 2; // Only 2 workers needed for simple counting + long loggingThreshold = calculateLoggingThreshold(table); - Table table = omMetadataManager.getTable(tableName); + LOG.info("{}: Processing simple table {} with parallel iteration ({} iterators, {} workers)", + getTaskName(), tableName, maxIterators, workerCount); - try (ParallelTableIteratorOperation parallelIter = - new ParallelTableIteratorOperation<>(omMetadataManager, table, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, 100000)) { + AtomicLong count = new AtomicLong(0); + + // Cast to String keys for parallel processing + Table genericTable = (Table) table; + + try (ParallelTableIteratorOperation parallelIter = + new ParallelTableIteratorOperation<>(omMetadataManager, genericTable, + StringCodec.get(), maxIterators, workerCount, maxKeysInMemory, loggingThreshold)) { - parallelIter.performTaskOnTableVals(tableName, null, null, kv -> { - if (kv != null && kv.getValue() != null) { - Object value = kv.getValue(); - - if (value instanceof OmKeyInfo) { - // For OPEN_KEY_TABLE and OPEN_FILE_TABLE - OmKeyInfo omKeyInfo = (OmKeyInfo) value; - unReplicatedSize.addAndGet(omKeyInfo.getDataSize()); - replicatedSize.addAndGet(omKeyInfo.getReplicatedSize()); - count.incrementAndGet(); - } else if (value instanceof RepeatedOmKeyInfo) { - // For DELETED_TABLE - match original getTableSizeAndCount() logic (lines 129-130) - RepeatedOmKeyInfo repeatedOmKeyInfo = (RepeatedOmKeyInfo) value; - org.apache.commons.lang3.tuple.Pair sizes = repeatedOmKeyInfo.getTotalSize(); - unReplicatedSize.addAndGet(sizes.getRight()); // Original: result.getRight() - replicatedSize.addAndGet(sizes.getLeft()); // Original: result.getLeft() - count.addAndGet(repeatedOmKeyInfo.getOmKeyInfoList().size()); - } + parallelIter.performTaskOnTableVals(getTaskName(), null, null, kv -> { + if (kv != null) { + count.incrementAndGet(); } return null; }); } - return Triple.of(count.get(), unReplicatedSize.get(), replicatedSize.get()); + long finalCount = count.get(); + LOG.info("{}: Table {} counted {} entries", getTaskName(), tableName, finalCount); + objectCountMap.put(getTableCountKeyFromTable(tableName), finalCount); + } + + /** + * Calculate logging threshold based on table size. + * Logs progress every 1% of total keys, minimum 1. + */ + private long calculateLoggingThreshold(Table table) { + try { + long estimatedKeys = table.getEstimatedKeyCount(); + return Math.max(estimatedKeys / 100, 1); + } catch (IOException e) { + LOG.debug("Could not estimate key count, using default logging threshold"); + return 100000; // Default: log every 100K keys + } } @Override From 8924135fce1b949e2f149e2ac8a20633180c0b76 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 26 Nov 2025 11:30:47 +0530 Subject: [PATCH 21/49] Small Refactors --- .../ozone/recon/tasks/OmTableInsightTask.java | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 91ef882949e5..6b30682450d0 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java @@ -127,16 +127,11 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { try { if (tableHandlers.containsKey(tableName)) { Table stringTable = (Table) table; - try (TableIterator> iterator - = stringTable.iterator()) { - Triple details = - tableHandlers.get(tableName).getTableSizeAndCount(iterator); - objectCountMap.put(getTableCountKeyFromTable(tableName), - details.getLeft()); - unReplicatedSizeMap.put( - getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); - replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), - details.getRight()); + try (TableIterator> iterator = stringTable.iterator()) { + Triple details = tableHandlers.get(tableName).getTableSizeAndCount(iterator); + objectCountMap.put(getTableCountKeyFromTable(tableName), details.getLeft()); + unReplicatedSizeMap.put(getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); + replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), details.getRight()); } } else { if (usesNonStringKeys(tableName)) { @@ -186,8 +181,7 @@ private boolean usesNonStringKeys(String tableName) { * Used for tables with non-String keys or as fallback. */ private void processTableSequentially(String tableName, Table table) throws IOException { - LOG.info("{}: Processing table {} sequentially (non-String keys)", - getTaskName(), tableName); + LOG.info("{}: Processing table {} sequentially (non-String keys)", getTaskName(), tableName); Table stringTable = (Table) table; try (TableIterator> iterator = stringTable.iterator()) { From 1787b65f86d3f180cc85ff2a0d31b8e596efd315 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 26 Nov 2025 12:42:42 +0530 Subject: [PATCH 22/49] Remove the flag isFlushingInProgress --- .../recon/tasks/ContainerKeyMapperHelper.java | 18 ++++-------- .../recon/tasks/FileSizeCountTaskHelper.java | 29 ++++++------------- 2 files changed, 14 insertions(+), 33 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 6d06fa10e11a..c238d2e53f0e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -30,7 +30,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -148,8 +147,6 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - // Flag to coordinate flush attempts - prevents all threads from queuing for write lock - AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); // Use parallel table iteration Function, Void> kvOperation = kv -> { @@ -163,18 +160,13 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, } omKeyCount.incrementAndGet(); - // Only one thread should attempt flush to avoid blocking all workers - if (localContainerKeyMap.size() >= containerKeyFlushToDBMaxThreshold && - isFlushingInProgress.compareAndSet(false, true)) { + // Check if flush is needed + if (localContainerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { try { lock.writeLock().lock(); - try { - if (!checkAndCallFlushToDB(localContainerKeyMap, containerKeyFlushToDBMaxThreshold, - reconContainerMetadataManager)) { - throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); - } - } finally { - isFlushingInProgress.set(false); // Reset flag after flush completes + if (!checkAndCallFlushToDB(localContainerKeyMap, containerKeyFlushToDBMaxThreshold, + reconContainerMetadataManager)) { + throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); } } finally { lock.writeLock().unlock(); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index 24e217d02170..b9bcb36a2524 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -21,7 +21,6 @@ import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; @@ -163,8 +162,6 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, // LOCAL lock (task-specific) - coordinates worker threads within this task only ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - // Flag to coordinate flush attempts - prevents all threads from queuing for write lock - AtomicBoolean isFlushingInProgress = new AtomicBoolean(false); final int FLUSH_THRESHOLD = 200000; // Use parallel table iteration @@ -176,26 +173,18 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, lock.readLock().unlock(); } - // Only one thread should attempt flush to avoid blocking all workers - if (fileSizeCountMap.size() >= FLUSH_THRESHOLD && - // If current value is false: Set to true and return true - // If current value is true: Do nothing and return false - isFlushingInProgress.compareAndSet(false, true)) { + // Check if flush is needed + if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { try { lock.writeLock().lock(); - try { - // Double-check size after acquiring write lock - if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { - LOG.debug("Flushing {} accumulated counts to RocksDB for {}", fileSizeCountMap.size(), taskName); - writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); - fileSizeCountMap.clear(); - } - } finally { - isFlushingInProgress.set(false); - lock.writeLock().unlock(); + // Double-check size after acquiring write lock + if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { + LOG.debug("Flushing {} accumulated counts to RocksDB for {}", fileSizeCountMap.size(), taskName); + writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); + fileSizeCountMap.clear(); } - } catch (Exception e) { - throw new RuntimeException("Failed to flush file size counts", e); + } finally { + lock.writeLock().unlock(); } } return null; From 37390f7ab04e0e7bbbce7715d05465d5fedde040 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 1 Dec 2025 09:10:07 +0530 Subject: [PATCH 23/49] Refactored changes 1 --- .../recon/tasks/FileSizeCountTaskHelper.java | 110 ++++++++++++------ .../ozone/recon/tasks/OmTableInsightTask.java | 10 +- 2 files changed, 79 insertions(+), 41 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index b9bcb36a2524..74deaa1a2487 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -17,11 +17,13 @@ package org.apache.hadoop.ozone.recon.tasks; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.hdds.utils.db.StringCodec; @@ -145,7 +147,8 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag } /** - * Iterates over the OM DB keys for the given bucket layout and updates the fileSizeCountMap (RocksDB version). + * Iterates over the OM DB keys for the given bucket layout using lockless per-worker maps. + * Each worker maintains its own map to eliminate read lock contention. */ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, OMMetadataManager omMetadataManager, @@ -155,59 +158,92 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, int maxIterators, int maxWorkers, int maxKeysInMemory) { - LOG.info("{}: Starting parallel iteration with {} iterators, {} workers for bucket layout {}", + LOG.info("{}: Starting lockless parallel iteration with {} iterators, {} workers for bucket layout {}", taskName, maxIterators, maxWorkers, bucketLayout); Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); long startTime = Time.monotonicNow(); - // LOCAL lock (task-specific) - coordinates worker threads within this task only - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - final int FLUSH_THRESHOLD = 200000; + // Per-worker threshold: divide total by worker count + final int PER_WORKER_THRESHOLD = Math.max(1, 200000 / maxWorkers); // 200k / 20 = 10k - // Use parallel table iteration + // Each worker thread gets its own map (lockless updates!) + //ThreadLocal> workerLocalMap = + // ThreadLocal.withInitial(ConcurrentHashMap::new); + + Map> allMap = new ConcurrentHashMap<>(); + // Track all worker maps for final flush + //List> allWorkerMaps = + // Collections.synchronizedList(new ArrayList<>()); + + // Single lock for DB flush operations only + Object flushLock = new Object(); + + // Worker operation - lockless map updates! Function, Void> kvOperation = kv -> { - try { - lock.readLock().lock(); - handlePutKeyEvent(kv.getValue(), fileSizeCountMap); - } finally { - lock.readLock().unlock(); - } - - // Check if flush is needed - if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { - try { - lock.writeLock().lock(); - // Double-check size after acquiring write lock - if (fileSizeCountMap.size() >= FLUSH_THRESHOLD) { - LOG.debug("Flushing {} accumulated counts to RocksDB for {}", fileSizeCountMap.size(), taskName); - writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); - fileSizeCountMap.clear(); - } - } finally { - lock.writeLock().unlock(); + // Get this worker thread's private map (no lock needed!) + //Map myMap = workerLocalMap.get(); + Map myMap = allMap.computeIfAbsent(Thread.currentThread().getId(), (k) -> new HashMap<>()); + //Map myMap = allMap.get(Thread.currentThread().getId()); + + // Register this worker's map on first use (thread-safe registration) + /*if (!allWorkerMaps.contains(myMap)) { + synchronized (allWorkerMaps) { + if (!allWorkerMaps.contains(myMap)) { + allWorkerMaps.add(myMap); + } + } + }*/ + + // Update map without any locks - each worker owns its map! + handlePutKeyEvent(kv.getValue(), myMap); + + // Check if this worker's map needs flushing + if (myMap.size() >= PER_WORKER_THRESHOLD) { + synchronized (flushLock) { + // Double-check size after acquiring lock + //if (myMap.size() >= PER_WORKER_THRESHOLD) { + LOG.info("{}: Worker flushing {} entries to RocksDB", + taskName, myMap.size()); + writeCountsToDB(myMap, reconFileMetadataManager); + myMap.clear(); + // } + } } - } - return null; + return null; }; try (ParallelTableIteratorOperation keyIter = new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, FLUSH_THRESHOLD)) { - keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, PER_WORKER_THRESHOLD)) { + keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } catch (Exception ex) { - LOG.error("Unable to populate File Size Count for {} in RocksDB.", taskName, ex); - return false; + LOG.error("Unable to populate File Size Count for {} in RocksDB.", taskName, ex); + return false; } + // Final flush: Ensure all worker maps are written to DB + // Since its a single thread flushing the maps one by one hence no need for the lock. + LOG.info("{}: Final flush of {} worker maps", taskName, allMap.size()); + //synchronized (flushLock) { + for (Map workerMap : allMap.values()) { + if (!workerMap.isEmpty()) { + LOG.info("{}: Flushing remaining {} entries from worker map", + taskName, workerMap.size()); + writeCountsToDB(workerMap, reconFileMetadataManager); + workerMap.clear(); + } + } + //} + + // Clean up ThreadLocal to prevent memory leaks + //workerLocalMap.remove(); + long endTime = Time.monotonicNow(); long durationMs = endTime - startTime; double durationSec = durationMs / 1000.0; - int totalKeys = fileSizeCountMap.values().stream().mapToInt(Long::intValue).sum(); - double throughput = totalKeys / Math.max(durationSec, 0.001); - LOG.info("{}: Reprocessed {} keys for bucket layout {} in {} ms ({} sec) - Throughput: {}/sec", - taskName, totalKeys, bucketLayout, durationMs, String.format("%.2f", durationSec), - String.format("%.2f", throughput)); + LOG.info("{}: Lockless parallel reprocess completed for {} in {} ms ({} sec) - Worker threshold: {} entries", + taskName, bucketLayout, durationMs, String.format("%.2f", durationSec), PER_WORKER_THRESHOLD); return true; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 6b30682450d0..3bff08f20f86 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java @@ -33,11 +33,13 @@ import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.tuple.Triple; +import org.apache.hadoop.hdds.utils.db.ByteArrayCodec; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.hdds.utils.db.cache.TableCache; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; @@ -135,7 +137,6 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { } } else { if (usesNonStringKeys(tableName)) { - processTableSequentially(tableName, table); } else { processTableInParallel(tableName, table, omMetadataManager); } @@ -207,9 +208,10 @@ private void processTableInParallel(String tableName, Table table, // Cast to String keys for parallel processing Table genericTable = (Table) table; - try (ParallelTableIteratorOperation parallelIter = - new ParallelTableIteratorOperation<>(omMetadataManager, genericTable, - StringCodec.get(), maxIterators, workerCount, maxKeysInMemory, loggingThreshold)) { + try (ParallelTableIteratorOperation parallelIter = new ParallelTableIteratorOperation<>( + omMetadataManager, omMetadataManager.getStore() + .getTable(tableName, StringCodec.get(), ByteArrayCodec.get(), TableCache.CacheType.NO_CACHE), StringCodec.get(), + maxIterators, workerCount, maxKeysInMemory, loggingThreshold)) { parallelIter.performTaskOnTableVals(getTaskName(), null, null, kv -> { if (kv != null) { From ce0d63b23d253005c2e267494d6bf9c2415eb91a Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 1 Dec 2025 10:06:59 +0530 Subject: [PATCH 24/49] Removed the complex locking in ContainerKeyMapper task --- .../ozone/recon/ReconControllerModule.java | 8 +- .../impl/OzoneManagerServiceProviderImpl.java | 280 +++++++++--------- .../recon/tasks/ContainerKeyMapperHelper.java | 79 ++--- .../recon/tasks/FileSizeCountTaskHelper.java | 68 ++--- 4 files changed, 209 insertions(+), 226 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java index 3f7e99056e44..d55cc6ed6b4e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java @@ -141,10 +141,10 @@ protected void configure() { Multibinder.newSetBinder(binder(), ReconOmTask.class); taskBinder.addBinding().to(ContainerKeyMapperTaskFSO.class); taskBinder.addBinding().to(ContainerKeyMapperTaskOBS.class); - taskBinder.addBinding().to(FileSizeCountTaskFSO.class); - taskBinder.addBinding().to(FileSizeCountTaskOBS.class); - taskBinder.addBinding().to(OmTableInsightTask.class); - taskBinder.addBinding().to(NSSummaryTask.class); +// taskBinder.addBinding().to(FileSizeCountTaskFSO.class); +// taskBinder.addBinding().to(FileSizeCountTaskOBS.class); +// taskBinder.addBinding().to(OmTableInsightTask.class); +// taskBinder.addBinding().to(NSSummaryTask.class); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 874049bfa89e..930768462c79 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -118,7 +118,7 @@ public class OzoneManagerServiceProviderImpl private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; - + private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); private File omSnapshotDBParentDir = null; private File reconDbDir = null; private String omDBSnapshotUrl; @@ -702,145 +702,153 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = false; + boolean fullSnapshot = true; + - if (currentSequenceNumber <= 0) { - fullSnapshot = true; + if (reInitializeTasksCalled.compareAndSet(false, true)) { + LOG.info("Calling reprocess on Recon tasks."); + reconTaskController.reInitializeTasks(omMetadataManager, null); } else { - // Get updates from OM and apply to local Recon OM DB and update task status in table - deltaReconTaskStatusUpdater.recordRunStart(); - int loopCount = 0; - long fromSequenceNumber = currentSequenceNumber; - long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; - /** - * This loop will continue to fetch and apply OM DB updates and with every - * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. - * It continues to fetch from OM till the lag, between OM DB WAL sequence number - * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. - * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. - */ - while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { - try (OMDBUpdatesHandler omdbUpdatesHandler = - new OMDBUpdatesHandler(omMetadataManager)) { - - // If interrupt was previously signalled, - // we should check for it before starting delta update sync. - if (Thread.currentThread().isInterrupted()) { - throw new InterruptedException("Thread interrupted during delta update."); - } - diffBetweenOMDbAndReconDBSeqNumber = - getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); - deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); - // Keeping last updated sequence number for both full and delta tasks to be same - // because sequence number of DB denotes and points to same OM DB copy of Recon, - // even though two different tasks are updating the DB at different conditions, but - // it tells the sync state with actual OM DB for the same Recon OM DB copy. - deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - deltaReconTaskStatusUpdater.recordRunCompletion(); - fullSnapshotReconTaskUpdater.updateDetails(); - // Update the current OM metadata manager in task controller - reconTaskController.updateOMMetadataManager(omMetadataManager); - - // Pass on DB update events to tasks that are listening. - reconTaskController.consumeOMEvents(new OMUpdateEventBatch( - omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); - - // Check if task reinitialization is needed due to buffer overflow or task failures - boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); - boolean tasksFailed = reconTaskController.hasTasksFailed(); - - if (bufferOverflowed || tasksFailed) { - ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? - ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : - ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; - - LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", - reason); - - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - - // Queue async reinitialization event - checkpoint creation and retry logic is handled internally - ReconTaskController.ReInitializationResult result = - reconTaskController.queueReInitializationEvent(reason); - - //TODO: Create a metric to track this event buffer overflow or task failure event - boolean triggerFullSnapshot = - Optional.ofNullable(result) - .map(r -> { - switch (r) { - case MAX_RETRIES_EXCEEDED: - LOG.warn( - "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + - "fallback"); - return true; - - case RETRY_LATER: - LOG.debug("Reinitialization event queueing will be retried in next iteration"); - return false; - - default: - LOG.info("Reinitialization event successfully queued"); - return false; - } - }) - .orElseGet(() -> { - LOG.error( - "ReInitializationResult is null, something went wrong in queueing reinitialization " + - "event"); - return true; - }); - - if (triggerFullSnapshot) { - fullSnapshot = true; - } - } - currentSequenceNumber = getCurrentOMDBSequenceNumber(); - LOG.debug("Updated current sequence number: {}", currentSequenceNumber); - loopCount++; - } catch (InterruptedException intEx) { - LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); - // We are updating the table even if it didn't run i.e. got interrupted beforehand - // to indicate that a task was supposed to run, but it didn't. - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - Thread.currentThread().interrupt(); - // Since thread is interrupted, we do not fall back to snapshot sync. - // Return with sync failed status. - return false; - } catch (Exception e) { - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", - e.getMessage()); - fullSnapshot = true; - } - if (fullSnapshot) { - break; - } - } - LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, - getCurrentOMDBSequenceNumber() - fromSequenceNumber); + LOG.info("reInitializeTasks already called once; skipping."); } - if (fullSnapshot) { - try { - executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); - } catch (InterruptedException intEx) { - LOG.error("OM DB Snapshot update sync thread was interrupted."); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - Thread.currentThread().interrupt(); - // Mark sync status as failed. - return false; - } catch (Exception e) { - metrics.incrNumSnapshotRequestsFailed(); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - LOG.error("Unable to update Recon's metadata with new OM DB. ", e); - // Update health status in ReconContext - reconContext.updateHealthStatus(new AtomicBoolean(false)); - reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); - } - } +// if (currentSequenceNumber <= 0) { +// fullSnapshot = true; +// } else { +// // Get updates from OM and apply to local Recon OM DB and update task status in table +// deltaReconTaskStatusUpdater.recordRunStart(); +// int loopCount = 0; +// long fromSequenceNumber = currentSequenceNumber; +// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; +// /** +// * This loop will continue to fetch and apply OM DB updates and with every +// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. +// * It continues to fetch from OM till the lag, between OM DB WAL sequence number +// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. +// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. +// */ +// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { +// try (OMDBUpdatesHandler omdbUpdatesHandler = +// new OMDBUpdatesHandler(omMetadataManager)) { +// +// // If interrupt was previously signalled, +// // we should check for it before starting delta update sync. +// if (Thread.currentThread().isInterrupted()) { +// throw new InterruptedException("Thread interrupted during delta update."); +// } +// diffBetweenOMDbAndReconDBSeqNumber = +// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); +// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); +// // Keeping last updated sequence number for both full and delta tasks to be same +// // because sequence number of DB denotes and points to same OM DB copy of Recon, +// // even though two different tasks are updating the DB at different conditions, but +// // it tells the sync state with actual OM DB for the same Recon OM DB copy. +// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// deltaReconTaskStatusUpdater.recordRunCompletion(); +// fullSnapshotReconTaskUpdater.updateDetails(); +// // Update the current OM metadata manager in task controller +// reconTaskController.updateOMMetadataManager(omMetadataManager); +// +// // Pass on DB update events to tasks that are listening. +// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( +// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); +// +// // Check if task reinitialization is needed due to buffer overflow or task failures +// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); +// boolean tasksFailed = reconTaskController.hasTasksFailed(); +// +// if (bufferOverflowed || tasksFailed) { +// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? +// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : +// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; +// +// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", +// reason); +// +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// +// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally +// ReconTaskController.ReInitializationResult result = +// reconTaskController.queueReInitializationEvent(reason); +// +// //TODO: Create a metric to track this event buffer overflow or task failure event +// boolean triggerFullSnapshot = +// Optional.ofNullable(result) +// .map(r -> { +// switch (r) { +// case MAX_RETRIES_EXCEEDED: +// LOG.warn( +// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + +// "fallback"); +// return true; +// +// case RETRY_LATER: +// LOG.debug("Reinitialization event queueing will be retried in next iteration"); +// return false; +// +// default: +// LOG.info("Reinitialization event successfully queued"); +// return false; +// } +// }) +// .orElseGet(() -> { +// LOG.error( +// "ReInitializationResult is null, something went wrong in queueing reinitialization " + +// "event"); +// return true; +// }); +// +// if (triggerFullSnapshot) { +// fullSnapshot = true; +// } +// } +// currentSequenceNumber = getCurrentOMDBSequenceNumber(); +// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); +// loopCount++; +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); +// // We are updating the table even if it didn't run i.e. got interrupted beforehand +// // to indicate that a task was supposed to run, but it didn't. +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// Thread.currentThread().interrupt(); +// // Since thread is interrupted, we do not fall back to snapshot sync. +// // Return with sync failed status. +// return false; +// } catch (Exception e) { +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", +// e.getMessage()); +// fullSnapshot = true; +// } +// if (fullSnapshot) { +// break; +// } +// } +// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, +// getCurrentOMDBSequenceNumber() - fromSequenceNumber); +// } + +// if (fullSnapshot) { +// try { +// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Snapshot update sync thread was interrupted."); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// Thread.currentThread().interrupt(); +// // Mark sync status as failed. +// return false; +// } catch (Exception e) { +// metrics.incrNumSnapshotRequestsFailed(); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); +// // Update health status in ReconContext +// reconContext.updateHealthStatus(new AtomicBoolean(false)); +// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); +// } +// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index c238d2e53f0e..fafc9876708c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -32,7 +32,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; @@ -133,43 +132,44 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, int maxWorkers, int maxKeysInMemory) { AtomicLong omKeyCount = new AtomicLong(0); - Map localContainerKeyMap = new ConcurrentHashMap<>(); try { - LOG.info("{}: Starting parallel reprocess with {} iterators, {} workers, max {} keys in memory for bucket layout {}", + LOG.info("{}: Starting lockless parallel reprocess with {} iterators, {} workers, max {} keys in memory for bucket layout {}", taskName, maxIterators, maxWorkers, maxKeysInMemory, bucketLayout); Instant start = Instant.now(); // Perform one-time initialization (truncate tables + clear shared map) initializeContainerKeyMapperIfNeeded(reconContainerMetadataManager, taskName); - // Get the appropriate table based on BucketLayout Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); - ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + // Divide threshold by worker count so each worker flushes independently + final long PER_WORKER_THRESHOLD = Math.max(1, containerKeyFlushToDBMaxThreshold / maxWorkers); + + // Map thread IDs to worker-specific local maps for lockless updates + Map> allLocalMaps = new ConcurrentHashMap<>(); + + Object flushLock = new Object(); - // Use parallel table iteration Function, Void> kvOperation = kv -> { try { - try { - lock.readLock().lock(); - handleKeyReprocess(kv.getKey(), kv.getValue(), localContainerKeyMap, SHARED_CONTAINER_KEY_COUNT_MAP, - reconContainerMetadataManager); - } finally { - lock.readLock().unlock(); - } + // Get or create this worker's private local map using thread ID + Map myLocalMap = allLocalMaps.computeIfAbsent( + Thread.currentThread().getId(), k -> new ConcurrentHashMap<>()); + + handleKeyReprocess(kv.getKey(), kv.getValue(), myLocalMap, SHARED_CONTAINER_KEY_COUNT_MAP, + reconContainerMetadataManager); + omKeyCount.incrementAndGet(); - // Check if flush is needed - if (localContainerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { - try { - lock.writeLock().lock(); - if (!checkAndCallFlushToDB(localContainerKeyMap, containerKeyFlushToDBMaxThreshold, + // Flush this worker's map when it reaches threshold + if (myLocalMap.size() >= PER_WORKER_THRESHOLD) { + synchronized (flushLock) { + LOG.info("{}: Worker flushing {} entries to RocksDB", taskName, myLocalMap.size()); + if (!flushAndCommitContainerKeyInfoToDB(myLocalMap, Collections.emptyMap(), reconContainerMetadataManager)) { throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); } - } finally { - lock.writeLock().unlock(); } } return null; @@ -177,18 +177,32 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, throw new UncheckedIOException(e); } }; + try (ParallelTableIteratorOperation keyIter = new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, containerKeyFlushToDBMaxThreshold)) { + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, PER_WORKER_THRESHOLD)) { keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } - // Capture total container count BEFORE final flush clears the map + // Final flush: Write remaining entries from all worker local maps to DB + LOG.info("{}: Final flush of {} worker local maps", taskName, allLocalMaps.size()); + for (Map workerLocalMap : allLocalMaps.values()) { + if (!workerLocalMap.isEmpty()) { + LOG.info("{}: Flushing remaining {} entries from worker map", taskName, workerLocalMap.size()); + if (!flushAndCommitContainerKeyInfoToDB(workerLocalMap, Collections.emptyMap(), + reconContainerMetadataManager)) { + LOG.error("Failed to flush worker local map for {}", taskName); + return false; + } + } + } + + // Capture total container count from shared map long totalContainers = SHARED_CONTAINER_KEY_COUNT_MAP.size(); - // Final flush and commit - if (!flushAndCommitContainerKeyInfoToDB(localContainerKeyMap, SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager)) { - LOG.error("Failed to flush Container Key data to DB for {}", taskName); + // Final flush: Shared container count map + if (!flushAndCommitContainerKeyInfoToDB(Collections.emptyMap(), SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager)) { + LOG.error("Failed to flush shared container count map for {}", taskName); return false; } @@ -216,10 +230,10 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, long keysProcessed = omKeyCount.get(); double throughput = keysProcessed / Math.max(durationSeconds, 0.001); - LOG.info("{}: Parallel reprocess completed. Processed {} keys in {} ms ({} sec) - " + - "Throughput: {} keys/sec - Containers: {}, Container-Key mappings: {}", + LOG.info("{}: Lockless parallel reprocess completed. Processed {} keys in {} ms ({} sec) - " + + "Throughput: {} keys/sec - Containers: {}, Worker threshold: {}", taskName, keysProcessed, durationMillis, String.format("%.2f", durationSeconds), - String.format("%.2f", throughput), totalContainers, localContainerKeyMap.size()); + String.format("%.2f", throughput), totalContainers, PER_WORKER_THRESHOLD); } catch (Exception ex) { LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ex); return false; @@ -227,15 +241,6 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, return true; } - private static boolean checkAndCallFlushToDB(Map localContainerKeyMap, - long containerKeyFlushToDBMaxThreshold, - ReconContainerMetadataManager reconContainerMetadataManager) { - if (localContainerKeyMap.size() >= containerKeyFlushToDBMaxThreshold) { - return flushAndCommitContainerKeyInfoToDB(localContainerKeyMap, Collections.emptyMap(), reconContainerMetadataManager); - } - return true; - } - public static boolean process(OMUpdateEventBatch events, String tableName, ReconContainerMetadataManager reconContainerMetadataManager, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index 74deaa1a2487..fd5c54e50df2 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -17,11 +17,8 @@ package org.apache.hadoop.ozone.recon.tasks; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; @@ -163,50 +160,30 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); long startTime = Time.monotonicNow(); - // Per-worker threshold: divide total by worker count + // Divide threshold by worker count so each worker flushes independently final int PER_WORKER_THRESHOLD = Math.max(1, 200000 / maxWorkers); // 200k / 20 = 10k - // Each worker thread gets its own map (lockless updates!) - //ThreadLocal> workerLocalMap = - // ThreadLocal.withInitial(ConcurrentHashMap::new); - + // Map thread IDs to worker-specific maps for lockless updates Map> allMap = new ConcurrentHashMap<>(); - // Track all worker maps for final flush - //List> allWorkerMaps = - // Collections.synchronizedList(new ArrayList<>()); - // Single lock for DB flush operations only + // Lock for coordinating DB flush operations only Object flushLock = new Object(); - // Worker operation - lockless map updates! + // Lambda executed by workers for each key Function, Void> kvOperation = kv -> { - // Get this worker thread's private map (no lock needed!) - //Map myMap = workerLocalMap.get(); - Map myMap = allMap.computeIfAbsent(Thread.currentThread().getId(), (k) -> new HashMap<>()); - //Map myMap = allMap.get(Thread.currentThread().getId()); - - // Register this worker's map on first use (thread-safe registration) - /*if (!allWorkerMaps.contains(myMap)) { - synchronized (allWorkerMaps) { - if (!allWorkerMaps.contains(myMap)) { - allWorkerMaps.add(myMap); - } - } - }*/ + // Get or create this worker's private map using thread ID + Map myMap = allMap.computeIfAbsent( + Thread.currentThread().getId(), k -> new HashMap<>()); - // Update map without any locks - each worker owns its map! + // Update worker's private map without locks handlePutKeyEvent(kv.getValue(), myMap); - // Check if this worker's map needs flushing + // Flush this worker's map when it reaches threshold if (myMap.size() >= PER_WORKER_THRESHOLD) { synchronized (flushLock) { - // Double-check size after acquiring lock - //if (myMap.size() >= PER_WORKER_THRESHOLD) { - LOG.info("{}: Worker flushing {} entries to RocksDB", - taskName, myMap.size()); - writeCountsToDB(myMap, reconFileMetadataManager); - myMap.clear(); - // } + LOG.info("{}: Worker flushing {} entries to RocksDB", taskName, myMap.size()); + writeCountsToDB(myMap, reconFileMetadataManager); + myMap.clear(); } } return null; @@ -221,22 +198,15 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, return false; } - // Final flush: Ensure all worker maps are written to DB - // Since its a single thread flushing the maps one by one hence no need for the lock. + // Final flush: Write remaining entries from all worker maps to DB LOG.info("{}: Final flush of {} worker maps", taskName, allMap.size()); - //synchronized (flushLock) { - for (Map workerMap : allMap.values()) { - if (!workerMap.isEmpty()) { - LOG.info("{}: Flushing remaining {} entries from worker map", - taskName, workerMap.size()); - writeCountsToDB(workerMap, reconFileMetadataManager); - workerMap.clear(); - } + for (Map workerMap : allMap.values()) { + if (!workerMap.isEmpty()) { + LOG.info("{}: Flushing remaining {} entries from worker map", taskName, workerMap.size()); + writeCountsToDB(workerMap, reconFileMetadataManager); + workerMap.clear(); } - //} - - // Clean up ThreadLocal to prevent memory leaks - //workerLocalMap.remove(); + } long endTime = Time.monotonicNow(); long durationMs = endTime - startTime; From 03b2dc299ffe860a8a9901e6eb4c11ce0490f356 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 1 Dec 2025 10:13:38 +0530 Subject: [PATCH 25/49] Added a Configurable flush threshold implemented --- .../hadoop/ozone/recon/ReconServerConfigKeys.java | 7 +++++++ .../hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java | 6 +++++- .../ozone/recon/tasks/FileSizeCountTaskHelper.java | 10 ++++++---- .../hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java | 6 +++++- 4 files changed, 23 insertions(+), 6 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java index 7c3bd6b8fcec..ba357f4ba145 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServerConfigKeys.java @@ -162,6 +162,13 @@ public final class ReconServerConfigKeys { public static final long OZONE_RECON_CONTAINER_KEY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT = 150 * 1000L; + public static final String + OZONE_RECON_FILESIZECOUNT_FLUSH_TO_DB_MAX_THRESHOLD = + "ozone.recon.filesizecount.flush.db.max.threshold"; + + public static final long + OZONE_RECON_FILESIZECOUNT_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT = 200 * 1000L; + public static final String OZONE_RECON_TASK_REPROCESS_MAX_ITERATORS = "ozone.recon.task.reprocess.max.iterators"; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java index 5f3118b3d675..229225de3385 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java @@ -63,6 +63,9 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { int maxWorkers = ozoneConfiguration.getInt( ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); + long fileSizeCountFlushThreshold = ozoneConfiguration.getLong( + ReconServerConfigKeys.OZONE_RECON_FILESIZECOUNT_FLUSH_TO_DB_MAX_THRESHOLD, + ReconServerConfigKeys.OZONE_RECON_FILESIZECOUNT_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); return FileSizeCountTaskHelper.reprocess( omMetadataManager, reconFileMetadataManager, @@ -70,7 +73,8 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { getTaskName(), maxIterators, maxWorkers, - maxKeysInMemory + maxKeysInMemory, + fileSizeCountFlushThreshold ); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index fd5c54e50df2..f13ff7a399f4 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -108,7 +108,8 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag String taskName, int maxIterators, int maxWorkers, - int maxKeysInMemory) { + int maxKeysInMemory, + long fileSizeCountFlushThreshold) { LOG.info("{}: Starting parallel RocksDB reprocess with {} iterators, {} workers for bucket layout {}", taskName, maxIterators, maxWorkers, bucketLayout); Map fileSizeCountMap = new ConcurrentHashMap<>(); @@ -120,7 +121,7 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag long iterationStartTime = Time.monotonicNow(); boolean status = reprocessBucketLayout( bucketLayout, omMetadataManager, fileSizeCountMap, reconFileMetadataManager, taskName, - maxIterators, maxWorkers, maxKeysInMemory); + maxIterators, maxWorkers, maxKeysInMemory, fileSizeCountFlushThreshold); if (!status) { return buildTaskResult(taskName, false); } @@ -154,14 +155,15 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, String taskName, int maxIterators, int maxWorkers, - int maxKeysInMemory) { + int maxKeysInMemory, + long fileSizeCountFlushThreshold) { LOG.info("{}: Starting lockless parallel iteration with {} iterators, {} workers for bucket layout {}", taskName, maxIterators, maxWorkers, bucketLayout); Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); long startTime = Time.monotonicNow(); // Divide threshold by worker count so each worker flushes independently - final int PER_WORKER_THRESHOLD = Math.max(1, 200000 / maxWorkers); // 200k / 20 = 10k + final long PER_WORKER_THRESHOLD = Math.max(1, fileSizeCountFlushThreshold / maxWorkers); // Map thread IDs to worker-specific maps for lockless updates Map> allMap = new ConcurrentHashMap<>(); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java index c4cf29b22e43..118edb6a1125 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java @@ -63,6 +63,9 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { int maxWorkers = ozoneConfiguration.getInt( ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS, ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); + long fileSizeCountFlushThreshold = ozoneConfiguration.getLong( + ReconServerConfigKeys.OZONE_RECON_FILESIZECOUNT_FLUSH_TO_DB_MAX_THRESHOLD, + ReconServerConfigKeys.OZONE_RECON_FILESIZECOUNT_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); return FileSizeCountTaskHelper.reprocess( omMetadataManager, reconFileMetadataManager, @@ -70,7 +73,8 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { getTaskName(), maxIterators, maxWorkers, - maxKeysInMemory + maxKeysInMemory, + fileSizeCountFlushThreshold ); } From 99098d9e2aab6478f0e41fa4bf7646dadaca3c50 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 1 Dec 2025 10:15:01 +0530 Subject: [PATCH 26/49] Removed unnecessary code from service provider impl --- .../impl/OzoneManagerServiceProviderImpl.java | 280 +++++++++--------- 1 file changed, 136 insertions(+), 144 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 930768462c79..874049bfa89e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -118,7 +118,7 @@ public class OzoneManagerServiceProviderImpl private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; - private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); + private File omSnapshotDBParentDir = null; private File reconDbDir = null; private String omDBSnapshotUrl; @@ -702,153 +702,145 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = true; - + boolean fullSnapshot = false; - if (reInitializeTasksCalled.compareAndSet(false, true)) { - LOG.info("Calling reprocess on Recon tasks."); - reconTaskController.reInitializeTasks(omMetadataManager, null); + if (currentSequenceNumber <= 0) { + fullSnapshot = true; } else { - LOG.info("reInitializeTasks already called once; skipping."); + // Get updates from OM and apply to local Recon OM DB and update task status in table + deltaReconTaskStatusUpdater.recordRunStart(); + int loopCount = 0; + long fromSequenceNumber = currentSequenceNumber; + long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; + /** + * This loop will continue to fetch and apply OM DB updates and with every + * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. + * It continues to fetch from OM till the lag, between OM DB WAL sequence number + * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. + * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. + */ + while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { + try (OMDBUpdatesHandler omdbUpdatesHandler = + new OMDBUpdatesHandler(omMetadataManager)) { + + // If interrupt was previously signalled, + // we should check for it before starting delta update sync. + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted during delta update."); + } + diffBetweenOMDbAndReconDBSeqNumber = + getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); + deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); + // Keeping last updated sequence number for both full and delta tasks to be same + // because sequence number of DB denotes and points to same OM DB copy of Recon, + // even though two different tasks are updating the DB at different conditions, but + // it tells the sync state with actual OM DB for the same Recon OM DB copy. + deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + deltaReconTaskStatusUpdater.recordRunCompletion(); + fullSnapshotReconTaskUpdater.updateDetails(); + // Update the current OM metadata manager in task controller + reconTaskController.updateOMMetadataManager(omMetadataManager); + + // Pass on DB update events to tasks that are listening. + reconTaskController.consumeOMEvents(new OMUpdateEventBatch( + omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); + + // Check if task reinitialization is needed due to buffer overflow or task failures + boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); + boolean tasksFailed = reconTaskController.hasTasksFailed(); + + if (bufferOverflowed || tasksFailed) { + ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? + ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : + ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; + + LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", + reason); + + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + + // Queue async reinitialization event - checkpoint creation and retry logic is handled internally + ReconTaskController.ReInitializationResult result = + reconTaskController.queueReInitializationEvent(reason); + + //TODO: Create a metric to track this event buffer overflow or task failure event + boolean triggerFullSnapshot = + Optional.ofNullable(result) + .map(r -> { + switch (r) { + case MAX_RETRIES_EXCEEDED: + LOG.warn( + "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + + "fallback"); + return true; + + case RETRY_LATER: + LOG.debug("Reinitialization event queueing will be retried in next iteration"); + return false; + + default: + LOG.info("Reinitialization event successfully queued"); + return false; + } + }) + .orElseGet(() -> { + LOG.error( + "ReInitializationResult is null, something went wrong in queueing reinitialization " + + "event"); + return true; + }); + + if (triggerFullSnapshot) { + fullSnapshot = true; + } + } + currentSequenceNumber = getCurrentOMDBSequenceNumber(); + LOG.debug("Updated current sequence number: {}", currentSequenceNumber); + loopCount++; + } catch (InterruptedException intEx) { + LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); + // We are updating the table even if it didn't run i.e. got interrupted beforehand + // to indicate that a task was supposed to run, but it didn't. + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + Thread.currentThread().interrupt(); + // Since thread is interrupted, we do not fall back to snapshot sync. + // Return with sync failed status. + return false; + } catch (Exception e) { + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", + e.getMessage()); + fullSnapshot = true; + } + if (fullSnapshot) { + break; + } + } + LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, + getCurrentOMDBSequenceNumber() - fromSequenceNumber); } -// if (currentSequenceNumber <= 0) { -// fullSnapshot = true; -// } else { -// // Get updates from OM and apply to local Recon OM DB and update task status in table -// deltaReconTaskStatusUpdater.recordRunStart(); -// int loopCount = 0; -// long fromSequenceNumber = currentSequenceNumber; -// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; -// /** -// * This loop will continue to fetch and apply OM DB updates and with every -// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. -// * It continues to fetch from OM till the lag, between OM DB WAL sequence number -// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. -// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. -// */ -// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { -// try (OMDBUpdatesHandler omdbUpdatesHandler = -// new OMDBUpdatesHandler(omMetadataManager)) { -// -// // If interrupt was previously signalled, -// // we should check for it before starting delta update sync. -// if (Thread.currentThread().isInterrupted()) { -// throw new InterruptedException("Thread interrupted during delta update."); -// } -// diffBetweenOMDbAndReconDBSeqNumber = -// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); -// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); -// // Keeping last updated sequence number for both full and delta tasks to be same -// // because sequence number of DB denotes and points to same OM DB copy of Recon, -// // even though two different tasks are updating the DB at different conditions, but -// // it tells the sync state with actual OM DB for the same Recon OM DB copy. -// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// deltaReconTaskStatusUpdater.recordRunCompletion(); -// fullSnapshotReconTaskUpdater.updateDetails(); -// // Update the current OM metadata manager in task controller -// reconTaskController.updateOMMetadataManager(omMetadataManager); -// -// // Pass on DB update events to tasks that are listening. -// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( -// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); -// -// // Check if task reinitialization is needed due to buffer overflow or task failures -// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); -// boolean tasksFailed = reconTaskController.hasTasksFailed(); -// -// if (bufferOverflowed || tasksFailed) { -// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? -// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : -// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; -// -// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", -// reason); -// -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// -// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally -// ReconTaskController.ReInitializationResult result = -// reconTaskController.queueReInitializationEvent(reason); -// -// //TODO: Create a metric to track this event buffer overflow or task failure event -// boolean triggerFullSnapshot = -// Optional.ofNullable(result) -// .map(r -> { -// switch (r) { -// case MAX_RETRIES_EXCEEDED: -// LOG.warn( -// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + -// "fallback"); -// return true; -// -// case RETRY_LATER: -// LOG.debug("Reinitialization event queueing will be retried in next iteration"); -// return false; -// -// default: -// LOG.info("Reinitialization event successfully queued"); -// return false; -// } -// }) -// .orElseGet(() -> { -// LOG.error( -// "ReInitializationResult is null, something went wrong in queueing reinitialization " + -// "event"); -// return true; -// }); -// -// if (triggerFullSnapshot) { -// fullSnapshot = true; -// } -// } -// currentSequenceNumber = getCurrentOMDBSequenceNumber(); -// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); -// loopCount++; -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); -// // We are updating the table even if it didn't run i.e. got interrupted beforehand -// // to indicate that a task was supposed to run, but it didn't. -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// Thread.currentThread().interrupt(); -// // Since thread is interrupted, we do not fall back to snapshot sync. -// // Return with sync failed status. -// return false; -// } catch (Exception e) { -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", -// e.getMessage()); -// fullSnapshot = true; -// } -// if (fullSnapshot) { -// break; -// } -// } -// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, -// getCurrentOMDBSequenceNumber() - fromSequenceNumber); -// } - -// if (fullSnapshot) { -// try { -// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Snapshot update sync thread was interrupted."); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// Thread.currentThread().interrupt(); -// // Mark sync status as failed. -// return false; -// } catch (Exception e) { -// metrics.incrNumSnapshotRequestsFailed(); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); -// // Update health status in ReconContext -// reconContext.updateHealthStatus(new AtomicBoolean(false)); -// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); -// } -// } + if (fullSnapshot) { + try { + executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); + } catch (InterruptedException intEx) { + LOG.error("OM DB Snapshot update sync thread was interrupted."); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + Thread.currentThread().interrupt(); + // Mark sync status as failed. + return false; + } catch (Exception e) { + metrics.incrNumSnapshotRequestsFailed(); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + LOG.error("Unable to update Recon's metadata with new OM DB. ", e); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); + } + } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); From e6a45526bc3218ae41e290f6363d03d4b7e3e6ce Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 1 Dec 2025 10:18:24 +0530 Subject: [PATCH 27/49] Removed unnecessary commits --- .../apache/hadoop/ozone/recon/ReconControllerModule.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java index d55cc6ed6b4e..3f7e99056e44 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java @@ -141,10 +141,10 @@ protected void configure() { Multibinder.newSetBinder(binder(), ReconOmTask.class); taskBinder.addBinding().to(ContainerKeyMapperTaskFSO.class); taskBinder.addBinding().to(ContainerKeyMapperTaskOBS.class); -// taskBinder.addBinding().to(FileSizeCountTaskFSO.class); -// taskBinder.addBinding().to(FileSizeCountTaskOBS.class); -// taskBinder.addBinding().to(OmTableInsightTask.class); -// taskBinder.addBinding().to(NSSummaryTask.class); + taskBinder.addBinding().to(FileSizeCountTaskFSO.class); + taskBinder.addBinding().to(FileSizeCountTaskOBS.class); + taskBinder.addBinding().to(OmTableInsightTask.class); + taskBinder.addBinding().to(NSSummaryTask.class); } } From c47a4ba44d7c43fe25f47e574a313479f0f29b74 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 1 Dec 2025 10:46:42 +0530 Subject: [PATCH 28/49] Removed unnecessary code --- .../recon/tasks/ContainerKeyMapperHelper.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index fafc9876708c..8e4e38b2efc1 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -131,7 +131,6 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, int maxIterators, int maxWorkers, int maxKeysInMemory) { - AtomicLong omKeyCount = new AtomicLong(0); try { LOG.info("{}: Starting lockless parallel reprocess with {} iterators, {} workers, max {} keys in memory for bucket layout {}", @@ -159,9 +158,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, handleKeyReprocess(kv.getKey(), kv.getValue(), myLocalMap, SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager); - - omKeyCount.incrementAndGet(); - + // Flush this worker's map when it reaches threshold if (myLocalMap.size() >= PER_WORKER_THRESHOLD) { synchronized (flushLock) { @@ -227,13 +224,10 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, Instant end = Instant.now(); long durationMillis = Duration.between(start, end).toMillis(); double durationSeconds = (double) durationMillis / 1000.0; - long keysProcessed = omKeyCount.get(); - double throughput = keysProcessed / Math.max(durationSeconds, 0.001); - - LOG.info("{}: Lockless parallel reprocess completed. Processed {} keys in {} ms ({} sec) - " + - "Throughput: {} keys/sec - Containers: {}, Worker threshold: {}", - taskName, keysProcessed, durationMillis, String.format("%.2f", durationSeconds), - String.format("%.2f", throughput), totalContainers, PER_WORKER_THRESHOLD); + + LOG.info("{}: Lockless parallel reprocess completed. Processed keys in {} ms ({} sec) - " + + "Containers: {}, Worker threshold: {}", + taskName, durationMillis, String.format("%.2f", durationSeconds), totalContainers, PER_WORKER_THRESHOLD); } catch (Exception ex) { LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ex); return false; From e795738181dca91d152feb987d27c7542e0dae7c Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 1 Dec 2025 14:33:43 +0530 Subject: [PATCH 29/49] Removed the unwanted change --- .../hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java index 4b98ddd80f09..f8b34f7f6d5c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java @@ -351,7 +351,7 @@ public synchronized void start() { // Clean up any pre-existing checkpoint directories from previous runs cleanupPreExistingCheckpoints(); - executorService = Executors.newFixedThreadPool(8, + executorService = Executors.newFixedThreadPool(threadCount, new ThreadFactoryBuilder().setNameFormat("ReconTaskThread-%d") .build()); From 34a8abc8072e69dff9cfdfa40fb8509717eb0963 Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 2 Dec 2025 12:03:30 +0530 Subject: [PATCH 30/49] Refactored the code and reduced the number of log messages --- .../recon/tasks/ContainerKeyMapperHelper.java | 15 ++------ .../recon/tasks/FileSizeCountTaskHelper.java | 34 ++++--------------- .../ozone/recon/tasks/OmTableInsightTask.java | 23 +++---------- 3 files changed, 14 insertions(+), 58 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 8e4e38b2efc1..a994acb38cf8 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -100,13 +100,10 @@ private static void initializeContainerKeyMapperIfNeeded( if (ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.compareAndSet(false, true)) { try { // Step 1: Truncate tables - LOG.info("{}: Truncating container key tables for reprocess", taskName); reconContainerMetadataManager.reinitWithNewContainerDataFromOm(Collections.emptyMap()); - LOG.info("{}: Successfully truncated container key tables", taskName); // Step 2: Clear shared map SHARED_CONTAINER_KEY_COUNT_MAP.clear(); - LOG.info("{}: Initialized shared container count map for cross-task synchronization", taskName); // Step 3: Initialize reference counter (2 tasks: FSO + OBS) ACTIVE_TASK_COUNT.set(2); @@ -133,8 +130,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, int maxKeysInMemory) { try { - LOG.info("{}: Starting lockless parallel reprocess with {} iterators, {} workers, max {} keys in memory for bucket layout {}", - taskName, maxIterators, maxWorkers, maxKeysInMemory, bucketLayout); + LOG.info("{}: Starting reprocess for bucket layout {}", taskName, bucketLayout); Instant start = Instant.now(); // Perform one-time initialization (truncate tables + clear shared map) @@ -162,7 +158,6 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Flush this worker's map when it reaches threshold if (myLocalMap.size() >= PER_WORKER_THRESHOLD) { synchronized (flushLock) { - LOG.info("{}: Worker flushing {} entries to RocksDB", taskName, myLocalMap.size()); if (!flushAndCommitContainerKeyInfoToDB(myLocalMap, Collections.emptyMap(), reconContainerMetadataManager)) { throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); @@ -182,10 +177,8 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, } // Final flush: Write remaining entries from all worker local maps to DB - LOG.info("{}: Final flush of {} worker local maps", taskName, allLocalMaps.size()); for (Map workerLocalMap : allLocalMaps.values()) { if (!workerLocalMap.isEmpty()) { - LOG.info("{}: Flushing remaining {} entries from worker map", taskName, workerLocalMap.size()); if (!flushAndCommitContainerKeyInfoToDB(workerLocalMap, Collections.emptyMap(), reconContainerMetadataManager)) { LOG.error("Failed to flush worker local map for {}", taskName); @@ -211,7 +204,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Decrement active task counter and cleanup if this is the last task int remainingTasks = ACTIVE_TASK_COUNT.decrementAndGet(); LOG.info("{}: Task completed. Remaining active tasks: {}", taskName, remainingTasks); - + if (remainingTasks == 0) { // Last task finished - clean up shared resources synchronized (INITIALIZATION_LOCK) { @@ -225,9 +218,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, long durationMillis = Duration.between(start, end).toMillis(); double durationSeconds = (double) durationMillis / 1000.0; - LOG.info("{}: Lockless parallel reprocess completed. Processed keys in {} ms ({} sec) - " + - "Containers: {}, Worker threshold: {}", - taskName, durationMillis, String.format("%.2f", durationSeconds), totalContainers, PER_WORKER_THRESHOLD); + LOG.info("{}: Reprocess completed in {} sec", taskName, durationSeconds); } catch (Exception ex) { LOG.error("Error populating Container Key data for {} in Recon DB.", taskName, ex); return false; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index f13ff7a399f4..d71543814362 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -110,36 +110,27 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag int maxWorkers, int maxKeysInMemory, long fileSizeCountFlushThreshold) { - LOG.info("{}: Starting parallel RocksDB reprocess with {} iterators, {} workers for bucket layout {}", - taskName, maxIterators, maxWorkers, bucketLayout); + LOG.info("{}: Starting reprocess for bucket layout {}", taskName, bucketLayout); Map fileSizeCountMap = new ConcurrentHashMap<>(); long overallStartTime = Time.monotonicNow(); // Ensure the file count table is truncated only once during reprocess truncateFileCountTableIfNeeded(reconFileMetadataManager, taskName); - long iterationStartTime = Time.monotonicNow(); boolean status = reprocessBucketLayout( bucketLayout, omMetadataManager, fileSizeCountMap, reconFileMetadataManager, taskName, maxIterators, maxWorkers, maxKeysInMemory, fileSizeCountFlushThreshold); if (!status) { return buildTaskResult(taskName, false); } - long iterationEndTime = Time.monotonicNow(); - long writeStartTime = Time.monotonicNow(); // Write remaining counts to DB (no global lock needed - FSO and OBS are mutually exclusive) writeCountsToDB(fileSizeCountMap, reconFileMetadataManager); - long writeEndTime = Time.monotonicNow(); - - long overallEndTime = Time.monotonicNow(); - long totalDurationMs = overallEndTime - overallStartTime; - long iterationDurationMs = iterationEndTime - iterationStartTime; - long writeDurationMs = writeEndTime - writeStartTime; - - LOG.info("{}: Parallel RocksDB reprocess completed - Total: {} ms (Iteration: {} ms, Write: {} ms) - " + - "File count entries: {}", taskName, totalDurationMs, iterationDurationMs, writeDurationMs, - fileSizeCountMap.size()); + + long totalDurationMs = Time.monotonicNow() - overallStartTime; + double durationSeconds = (double) totalDurationMs / 1000.0; + + LOG.info("{}: Reprocess completed in {} sec", taskName, durationSeconds); return buildTaskResult(taskName, true); } @@ -157,10 +148,7 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, int maxWorkers, int maxKeysInMemory, long fileSizeCountFlushThreshold) { - LOG.info("{}: Starting lockless parallel iteration with {} iterators, {} workers for bucket layout {}", - taskName, maxIterators, maxWorkers, bucketLayout); Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); - long startTime = Time.monotonicNow(); // Divide threshold by worker count so each worker flushes independently final long PER_WORKER_THRESHOLD = Math.max(1, fileSizeCountFlushThreshold / maxWorkers); @@ -183,7 +171,6 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, // Flush this worker's map when it reaches threshold if (myMap.size() >= PER_WORKER_THRESHOLD) { synchronized (flushLock) { - LOG.info("{}: Worker flushing {} entries to RocksDB", taskName, myMap.size()); writeCountsToDB(myMap, reconFileMetadataManager); myMap.clear(); } @@ -201,22 +188,13 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, } // Final flush: Write remaining entries from all worker maps to DB - LOG.info("{}: Final flush of {} worker maps", taskName, allMap.size()); for (Map workerMap : allMap.values()) { if (!workerMap.isEmpty()) { - LOG.info("{}: Flushing remaining {} entries from worker map", taskName, workerMap.size()); writeCountsToDB(workerMap, reconFileMetadataManager); workerMap.clear(); } } - long endTime = Time.monotonicNow(); - long durationMs = endTime - startTime; - double durationSec = durationMs / 1000.0; - - LOG.info("{}: Lockless parallel reprocess completed for {} in {} ms ({} sec) - Worker threshold: {} entries", - taskName, bucketLayout, durationMs, String.format("%.2f", durationSec), PER_WORKER_THRESHOLD); - return true; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 3bff08f20f86..58854cf90919 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java @@ -119,7 +119,7 @@ public void init() { */ @Override public TaskResult reprocess(OMMetadataManager omMetadataManager) { - LOG.info("Starting RocksDB Reprocess for {}", getTaskName()); + LOG.info("{}: Starting reprocess", getTaskName()); long startTime = Time.monotonicNow(); init(); @@ -137,6 +137,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { } } else { if (usesNonStringKeys(tableName)) { + processTableSequentially(tableName, table); } else { processTableInParallel(tableName, table, omMetadataManager); } @@ -158,14 +159,8 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { } long endTime = Time.monotonicNow(); long durationMs = endTime - startTime; - double durationSec = durationMs / 1000.0; - int handlerTables = tableHandlers.size(); // 3 handler tables (with size calculation) - int simpleTables = tables.size() - handlerTables; // Simple count-only tables - - LOG.info("{}: RocksDB reprocess completed in {} ms ({} sec) - " + - "Total tables: {}, Handler tables (with size): {}, Simple tables (count only): {}", - getTaskName(), durationMs, String.format("%.2f", durationSec), - tables.size(), handlerTables, simpleTables); + + LOG.info("{}: Reprocess completed in {} ms", getTaskName(), durationMs); return buildTaskResult(true); } @@ -200,13 +195,7 @@ private void processTableInParallel(String tableName, Table table, int workerCount = 2; // Only 2 workers needed for simple counting long loggingThreshold = calculateLoggingThreshold(table); - LOG.info("{}: Processing simple table {} with parallel iteration ({} iterators, {} workers)", - getTaskName(), tableName, maxIterators, workerCount); - AtomicLong count = new AtomicLong(0); - - // Cast to String keys for parallel processing - Table genericTable = (Table) table; try (ParallelTableIteratorOperation parallelIter = new ParallelTableIteratorOperation<>( omMetadataManager, omMetadataManager.getStore() @@ -221,9 +210,7 @@ private void processTableInParallel(String tableName, Table table, }); } - long finalCount = count.get(); - LOG.info("{}: Table {} counted {} entries", getTaskName(), tableName, finalCount); - objectCountMap.put(getTableCountKeyFromTable(tableName), finalCount); + objectCountMap.put(getTableCountKeyFromTable(tableName), count.get()); } /** From eccd3704c61153f006e0ff98fe63c2ee49727041 Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 4 Dec 2025 12:14:35 +0530 Subject: [PATCH 31/49] Fixed final review comments --- .../ozone/recon/ReconControllerModule.java | 10 +- .../impl/OzoneManagerServiceProviderImpl.java | 279 +++++++++--------- .../recon/tasks/ContainerKeyMapperHelper.java | 60 ++-- .../tasks/DeletedKeysInsightHandler.java | 9 +- .../recon/tasks/FileSizeCountTaskHelper.java | 18 +- .../ozone/recon/tasks/OmTableHandler.java | 14 +- .../ozone/recon/tasks/OmTableInsightTask.java | 61 ++-- .../recon/tasks/OpenKeysInsightHandler.java | 9 +- 8 files changed, 233 insertions(+), 227 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java index 3f7e99056e44..cea34593ed44 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java @@ -139,12 +139,12 @@ static class ReconOmTaskBindingModule extends AbstractModule { protected void configure() { Multibinder taskBinder = Multibinder.newSetBinder(binder(), ReconOmTask.class); - taskBinder.addBinding().to(ContainerKeyMapperTaskFSO.class); - taskBinder.addBinding().to(ContainerKeyMapperTaskOBS.class); - taskBinder.addBinding().to(FileSizeCountTaskFSO.class); - taskBinder.addBinding().to(FileSizeCountTaskOBS.class); +// taskBinder.addBinding().to(ContainerKeyMapperTaskFSO.class); +// taskBinder.addBinding().to(ContainerKeyMapperTaskOBS.class); +// taskBinder.addBinding().to(FileSizeCountTaskFSO.class); +// taskBinder.addBinding().to(FileSizeCountTaskOBS.class); taskBinder.addBinding().to(OmTableInsightTask.class); - taskBinder.addBinding().to(NSSummaryTask.class); +// taskBinder.addBinding().to(NSSummaryTask.class); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 874049bfa89e..8b61b3d549ff 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -143,6 +143,8 @@ public class OzoneManagerServiceProviderImpl private ReconTaskStatusUpdaterManager taskStatusUpdaterManager; private TarExtractor tarExtractor; + private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); + /** * OM Snapshot related task names. */ @@ -702,145 +704,152 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = false; + boolean fullSnapshot = true; - if (currentSequenceNumber <= 0) { - fullSnapshot = true; + if (reInitializeTasksCalled.compareAndSet(false, true)) { + LOG.info("Calling reprocess on Recon tasks."); + reconTaskController.reInitializeTasks(omMetadataManager, null); } else { - // Get updates from OM and apply to local Recon OM DB and update task status in table - deltaReconTaskStatusUpdater.recordRunStart(); - int loopCount = 0; - long fromSequenceNumber = currentSequenceNumber; - long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; - /** - * This loop will continue to fetch and apply OM DB updates and with every - * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. - * It continues to fetch from OM till the lag, between OM DB WAL sequence number - * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. - * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. - */ - while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { - try (OMDBUpdatesHandler omdbUpdatesHandler = - new OMDBUpdatesHandler(omMetadataManager)) { - - // If interrupt was previously signalled, - // we should check for it before starting delta update sync. - if (Thread.currentThread().isInterrupted()) { - throw new InterruptedException("Thread interrupted during delta update."); - } - diffBetweenOMDbAndReconDBSeqNumber = - getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); - deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); - // Keeping last updated sequence number for both full and delta tasks to be same - // because sequence number of DB denotes and points to same OM DB copy of Recon, - // even though two different tasks are updating the DB at different conditions, but - // it tells the sync state with actual OM DB for the same Recon OM DB copy. - deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - deltaReconTaskStatusUpdater.recordRunCompletion(); - fullSnapshotReconTaskUpdater.updateDetails(); - // Update the current OM metadata manager in task controller - reconTaskController.updateOMMetadataManager(omMetadataManager); - - // Pass on DB update events to tasks that are listening. - reconTaskController.consumeOMEvents(new OMUpdateEventBatch( - omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); - - // Check if task reinitialization is needed due to buffer overflow or task failures - boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); - boolean tasksFailed = reconTaskController.hasTasksFailed(); - - if (bufferOverflowed || tasksFailed) { - ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? - ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : - ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; - - LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", - reason); - - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - - // Queue async reinitialization event - checkpoint creation and retry logic is handled internally - ReconTaskController.ReInitializationResult result = - reconTaskController.queueReInitializationEvent(reason); - - //TODO: Create a metric to track this event buffer overflow or task failure event - boolean triggerFullSnapshot = - Optional.ofNullable(result) - .map(r -> { - switch (r) { - case MAX_RETRIES_EXCEEDED: - LOG.warn( - "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + - "fallback"); - return true; - - case RETRY_LATER: - LOG.debug("Reinitialization event queueing will be retried in next iteration"); - return false; - - default: - LOG.info("Reinitialization event successfully queued"); - return false; - } - }) - .orElseGet(() -> { - LOG.error( - "ReInitializationResult is null, something went wrong in queueing reinitialization " + - "event"); - return true; - }); - - if (triggerFullSnapshot) { - fullSnapshot = true; - } - } - currentSequenceNumber = getCurrentOMDBSequenceNumber(); - LOG.debug("Updated current sequence number: {}", currentSequenceNumber); - loopCount++; - } catch (InterruptedException intEx) { - LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); - // We are updating the table even if it didn't run i.e. got interrupted beforehand - // to indicate that a task was supposed to run, but it didn't. - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - Thread.currentThread().interrupt(); - // Since thread is interrupted, we do not fall back to snapshot sync. - // Return with sync failed status. - return false; - } catch (Exception e) { - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", - e.getMessage()); - fullSnapshot = true; - } - if (fullSnapshot) { - break; - } - } - LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, - getCurrentOMDBSequenceNumber() - fromSequenceNumber); + LOG.info("reInitializeTasks already called once; skipping."); } - if (fullSnapshot) { - try { - executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); - } catch (InterruptedException intEx) { - LOG.error("OM DB Snapshot update sync thread was interrupted."); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - Thread.currentThread().interrupt(); - // Mark sync status as failed. - return false; - } catch (Exception e) { - metrics.incrNumSnapshotRequestsFailed(); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - LOG.error("Unable to update Recon's metadata with new OM DB. ", e); - // Update health status in ReconContext - reconContext.updateHealthStatus(new AtomicBoolean(false)); - reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); - } - } +// if (currentSequenceNumber <= 0) { +// fullSnapshot = true; +// } else { +// // Get updates from OM and apply to local Recon OM DB and update task status in table +// deltaReconTaskStatusUpdater.recordRunStart(); +// int loopCount = 0; +// long fromSequenceNumber = currentSequenceNumber; +// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; +// /** +// * This loop will continue to fetch and apply OM DB updates and with every +// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. +// * It continues to fetch from OM till the lag, between OM DB WAL sequence number +// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. +// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. +// */ +// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { +// try (OMDBUpdatesHandler omdbUpdatesHandler = +// new OMDBUpdatesHandler(omMetadataManager)) { +// +// // If interrupt was previously signalled, +// // we should check for it before starting delta update sync. +// if (Thread.currentThread().isInterrupted()) { +// throw new InterruptedException("Thread interrupted during delta update."); +// } +// diffBetweenOMDbAndReconDBSeqNumber = +// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); +// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); +// // Keeping last updated sequence number for both full and delta tasks to be same +// // because sequence number of DB denotes and points to same OM DB copy of Recon, +// // even though two different tasks are updating the DB at different conditions, but +// // it tells the sync state with actual OM DB for the same Recon OM DB copy. +// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// deltaReconTaskStatusUpdater.recordRunCompletion(); +// fullSnapshotReconTaskUpdater.updateDetails(); +// // Update the current OM metadata manager in task controller +// reconTaskController.updateOMMetadataManager(omMetadataManager); +// +// // Pass on DB update events to tasks that are listening. +// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( +// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); +// +// // Check if task reinitialization is needed due to buffer overflow or task failures +// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); +// boolean tasksFailed = reconTaskController.hasTasksFailed(); +// +// if (bufferOverflowed || tasksFailed) { +// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? +// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : +// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; +// +// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", +// reason); +// +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// +// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally +// ReconTaskController.ReInitializationResult result = +// reconTaskController.queueReInitializationEvent(reason); +// +// //TODO: Create a metric to track this event buffer overflow or task failure event +// boolean triggerFullSnapshot = +// Optional.ofNullable(result) +// .map(r -> { +// switch (r) { +// case MAX_RETRIES_EXCEEDED: +// LOG.warn( +// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + +// "fallback"); +// return true; +// +// case RETRY_LATER: +// LOG.debug("Reinitialization event queueing will be retried in next iteration"); +// return false; +// +// default: +// LOG.info("Reinitialization event successfully queued"); +// return false; +// } +// }) +// .orElseGet(() -> { +// LOG.error( +// "ReInitializationResult is null, something went wrong in queueing reinitialization " + +// "event"); +// return true; +// }); +// +// if (triggerFullSnapshot) { +// fullSnapshot = true; +// } +// } +// currentSequenceNumber = getCurrentOMDBSequenceNumber(); +// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); +// loopCount++; +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); +// // We are updating the table even if it didn't run i.e. got interrupted beforehand +// // to indicate that a task was supposed to run, but it didn't. +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// Thread.currentThread().interrupt(); +// // Since thread is interrupted, we do not fall back to snapshot sync. +// // Return with sync failed status. +// return false; +// } catch (Exception e) { +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", +// e.getMessage()); +// fullSnapshot = true; +// } +// if (fullSnapshot) { +// break; +// } +// } +// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, +// getCurrentOMDBSequenceNumber() - fromSequenceNumber); +// } + +// if (fullSnapshot) { +// try { +// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Snapshot update sync thread was interrupted."); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// Thread.currentThread().interrupt(); +// // Mark sync status as failed. +// return false; +// } catch (Exception e) { +// metrics.incrNumSnapshotRequestsFailed(); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); +// // Update health status in ReconContext +// reconContext.updateHealthStatus(new AtomicBoolean(false)); +// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); +// } +// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index a994acb38cf8..de8b6628fda7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -105,8 +105,8 @@ private static void initializeContainerKeyMapperIfNeeded( // Step 2: Clear shared map SHARED_CONTAINER_KEY_COUNT_MAP.clear(); - // Step 3: Initialize reference counter (2 tasks: FSO + OBS) - ACTIVE_TASK_COUNT.set(2); + // Step 3: Increment the refrence counter for active tasks + ACTIVE_TASK_COUNT.incrementAndGet(); } catch (Exception e) { // CRITICAL: Reset flag so another task can retry @@ -144,24 +144,20 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Map thread IDs to worker-specific local maps for lockless updates Map> allLocalMaps = new ConcurrentHashMap<>(); - Object flushLock = new Object(); - Function, Void> kvOperation = kv -> { try { // Get or create this worker's private local map using thread ID - Map myLocalMap = allLocalMaps.computeIfAbsent( + Map containerKeyPrefixMap = allLocalMaps.computeIfAbsent( Thread.currentThread().getId(), k -> new ConcurrentHashMap<>()); - handleKeyReprocess(kv.getKey(), kv.getValue(), myLocalMap, SHARED_CONTAINER_KEY_COUNT_MAP, + handleKeyReprocess(kv.getKey(), kv.getValue(), containerKeyPrefixMap, SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager); // Flush this worker's map when it reaches threshold - if (myLocalMap.size() >= PER_WORKER_THRESHOLD) { - synchronized (flushLock) { - if (!flushAndCommitContainerKeyInfoToDB(myLocalMap, Collections.emptyMap(), - reconContainerMetadataManager)) { - throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); - } + if (containerKeyPrefixMap.size() >= PER_WORKER_THRESHOLD) { + if (!flushAndCommitContainerKeyInfoToDB(containerKeyPrefixMap, Collections.emptyMap(), + reconContainerMetadataManager)) { + throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); } } return null; @@ -177,9 +173,9 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, } // Final flush: Write remaining entries from all worker local maps to DB - for (Map workerLocalMap : allLocalMaps.values()) { - if (!workerLocalMap.isEmpty()) { - if (!flushAndCommitContainerKeyInfoToDB(workerLocalMap, Collections.emptyMap(), + for (Map containerKeyPrefixMap : allLocalMaps.values()) { + if (!containerKeyPrefixMap.isEmpty()) { + if (!flushAndCommitContainerKeyInfoToDB(containerKeyPrefixMap, Collections.emptyMap(), reconContainerMetadataManager)) { LOG.error("Failed to flush worker local map for {}", taskName); return false; @@ -187,27 +183,29 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, } } - // Capture total container count from shared map - long totalContainers = SHARED_CONTAINER_KEY_COUNT_MAP.size(); - - // Final flush: Shared container count map - if (!flushAndCommitContainerKeyInfoToDB(Collections.emptyMap(), SHARED_CONTAINER_KEY_COUNT_MAP, reconContainerMetadataManager)) { - LOG.error("Failed to flush shared container count map for {}", taskName); - return false; - } - - // Write total container count once at the end (after all processing) - if (totalContainers > 0) { - reconContainerMetadataManager.incrementContainerCountBy(totalContainers); - } - - // Decrement active task counter and cleanup if this is the last task + // Decrement active task counter int remainingTasks = ACTIVE_TASK_COUNT.decrementAndGet(); LOG.info("{}: Task completed. Remaining active tasks: {}", taskName, remainingTasks); + // Only last task flushes shared map and writes container count if (remainingTasks == 0) { - // Last task finished - clean up shared resources synchronized (INITIALIZATION_LOCK) { + // Capture total container count from shared map + long totalContainers = SHARED_CONTAINER_KEY_COUNT_MAP.size(); + + // Flush shared container count map + if (!flushAndCommitContainerKeyInfoToDB(Collections.emptyMap(), SHARED_CONTAINER_KEY_COUNT_MAP, + reconContainerMetadataManager)) { + LOG.error("Failed to flush shared container count map for {}", taskName); + return false; + } + + // Write total container count once at the end + if (totalContainers > 0) { + reconContainerMetadataManager.incrementContainerCountBy(totalContainers); + } + + // Clean up shared resources SHARED_CONTAINER_KEY_COUNT_MAP.clear(); ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); LOG.debug("{}: Last task completed. Cleared shared map and reset initialization flag.", taskName); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java index f315ddf3e862..43386c556f9e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java @@ -23,6 +23,7 @@ import org.apache.commons.lang3.tuple.Triple; 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.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,14 +113,14 @@ public void handleUpdateEvent(OMDBUpdateEvent event, * pending deletion in Ozone. */ @Override - public Triple getTableSizeAndCount( - TableIterator> iterator) - throws IOException { + public Triple getTableSizeAndCount(String tableName, + OMMetadataManager omMetadataManager) throws IOException { long count = 0; long unReplicatedSize = 0; long replicatedSize = 0; - if (iterator != null) { + Table table = (Table) omMetadataManager.getTable(tableName); + try (TableIterator> iterator = table.iterator()) { while (iterator.hasNext()) { Table.KeyValue kv = iterator.next(); if (kv != null && kv.getValue() != null) { diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index d71543814362..dd3a974198b7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -162,17 +162,17 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, // Lambda executed by workers for each key Function, Void> kvOperation = kv -> { // Get or create this worker's private map using thread ID - Map myMap = allMap.computeIfAbsent( + Map workerFileSizeCountMap = allMap.computeIfAbsent( Thread.currentThread().getId(), k -> new HashMap<>()); // Update worker's private map without locks - handlePutKeyEvent(kv.getValue(), myMap); + handlePutKeyEvent(kv.getValue(), workerFileSizeCountMap); // Flush this worker's map when it reaches threshold - if (myMap.size() >= PER_WORKER_THRESHOLD) { + if (workerFileSizeCountMap.size() >= PER_WORKER_THRESHOLD) { synchronized (flushLock) { - writeCountsToDB(myMap, reconFileMetadataManager); - myMap.clear(); + writeCountsToDB(workerFileSizeCountMap, reconFileMetadataManager); + workerFileSizeCountMap.clear(); } } return null; @@ -188,10 +188,10 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, } // Final flush: Write remaining entries from all worker maps to DB - for (Map workerMap : allMap.values()) { - if (!workerMap.isEmpty()) { - writeCountsToDB(workerMap, reconFileMetadataManager); - workerMap.clear(); + for (Map workerFileSizeCountMap : allMap.values()) { + if (!workerFileSizeCountMap.isEmpty()) { + writeCountsToDB(workerFileSizeCountMap, reconFileMetadataManager); + workerFileSizeCountMap.clear(); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableHandler.java index 401a3c7dc099..7e6fa5b9fc89 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableHandler.java @@ -20,8 +20,7 @@ import java.io.IOException; import java.util.Map; import org.apache.commons.lang3.tuple.Triple; -import org.apache.hadoop.hdds.utils.db.Table; -import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.om.OMMetadataManager; /** * Interface for handling PUT, DELETE and UPDATE events for size-related @@ -82,19 +81,18 @@ void handleUpdateEvent(OMDBUpdateEvent event, /** * Returns a triple with the total count of records (left), total unreplicated - * size (middle), and total replicated size (right) in the given iterator. + * size (middle), and total replicated size (right) for the given table. * Increments count for each record and adds the dataSize if a record's value * is an instance of OmKeyInfo,RepeatedOmKeyInfo. - * If the iterator is null, returns (0, 0, 0). * - * @param iterator The iterator over the table to be iterated. + * @param tableName The name of the table to process. + * @param omMetadataManager The OM metadata manager to get the table. * @return A Triple with three Long values representing the count, * unReplicated size and replicated size. * @throws IOException If an I/O error occurs during the iterator traversal. */ - Triple getTableSizeAndCount( - TableIterator> iterator) - throws IOException; + Triple getTableSizeAndCount(String tableName, + OMMetadataManager omMetadataManager) throws IOException; /** * Returns the count key for the given table. diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 58854cf90919..40e40cba85c5 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java @@ -124,22 +124,18 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { init(); for (String tableName : tables) { - Table table = omMetadataManager.getTable(tableName); - try { if (tableHandlers.containsKey(tableName)) { - Table stringTable = (Table) table; - try (TableIterator> iterator = stringTable.iterator()) { - Triple details = tableHandlers.get(tableName).getTableSizeAndCount(iterator); - objectCountMap.put(getTableCountKeyFromTable(tableName), details.getLeft()); - unReplicatedSizeMap.put(getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); - replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), details.getRight()); - } + Triple details = + tableHandlers.get(tableName).getTableSizeAndCount(tableName, omMetadataManager); + objectCountMap.put(getTableCountKeyFromTable(tableName), details.getLeft()); + unReplicatedSizeMap.put(getUnReplicatedSizeKeyFromTable(tableName), details.getMiddle()); + replicatedSizeMap.put(getReplicatedSizeKeyFromTable(tableName), details.getRight()); } else { if (usesNonStringKeys(tableName)) { - processTableSequentially(tableName, table); + processTableSequentially(tableName, omMetadataManager); } else { - processTableInParallel(tableName, table, omMetadataManager); + processTableInParallel(tableName, omMetadataManager); } } } catch (Exception ioEx) { @@ -173,15 +169,16 @@ private boolean usesNonStringKeys(String tableName) { } /** - * Process table sequentially using raw iterator (no type assumptions). + * Process table sequentially using key-only iterator. * Used for tables with non-String keys or as fallback. */ - private void processTableSequentially(String tableName, Table table) throws IOException { + private void processTableSequentially(String tableName, OMMetadataManager omMetadataManager) throws IOException { LOG.info("{}: Processing table {} sequentially (non-String keys)", getTaskName(), tableName); - - Table stringTable = (Table) table; - try (TableIterator> iterator = stringTable.iterator()) { - long count = Iterators.size(iterator); + + Table table = omMetadataManager.getStore() + .getTable(tableName, ByteArrayCodec.get(), ByteArrayCodec.get(), TableCache.CacheType.NO_CACHE); + try (TableIterator keyIterator = table.keyIterator()) { + long count = Iterators.size(keyIterator); objectCountMap.put(getTableCountKeyFromTable(tableName), count); } } @@ -190,16 +187,24 @@ private void processTableSequentially(String tableName, Table table) throw * Process table in parallel using multiple iterators and workers. * Only for tables with String keys. */ - private void processTableInParallel(String tableName, Table table, - OMMetadataManager omMetadataManager) throws Exception { + private void processTableInParallel(String tableName, OMMetadataManager omMetadataManager) throws Exception { int workerCount = 2; // Only 2 workers needed for simple counting - long loggingThreshold = calculateLoggingThreshold(table); + + Table table = omMetadataManager.getStore() + .getTable(tableName, StringCodec.get(), ByteArrayCodec.get(), TableCache.CacheType.NO_CACHE); + + long estimatedCount = 100000; // Default + try { + estimatedCount = table.getEstimatedKeyCount(); + } catch (IOException e) { + LOG.info("Could not estimate key count for table {}, using default", tableName); + } + long loggingThreshold = calculateLoggingThreshold(estimatedCount); AtomicLong count = new AtomicLong(0); try (ParallelTableIteratorOperation parallelIter = new ParallelTableIteratorOperation<>( - omMetadataManager, omMetadataManager.getStore() - .getTable(tableName, StringCodec.get(), ByteArrayCodec.get(), TableCache.CacheType.NO_CACHE), StringCodec.get(), + omMetadataManager, table, StringCodec.get(), maxIterators, workerCount, maxKeysInMemory, loggingThreshold)) { parallelIter.performTaskOnTableVals(getTaskName(), null, null, kv -> { @@ -214,17 +219,11 @@ private void processTableInParallel(String tableName, Table table, } /** - * Calculate logging threshold based on table size. + * Calculate logging threshold based on estimated key count. * Logs progress every 1% of total keys, minimum 1. */ - private long calculateLoggingThreshold(Table table) { - try { - long estimatedKeys = table.getEstimatedKeyCount(); - return Math.max(estimatedKeys / 100, 1); - } catch (IOException e) { - LOG.debug("Could not estimate key count, using default logging threshold"); - return 100000; // Default: log every 100K keys - } + private long calculateLoggingThreshold(long estimatedCount) { + return Math.max(estimatedCount / 100, 1); } @Override diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java index d22963ed2807..6264ad3b2b4a 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java @@ -22,6 +22,7 @@ import org.apache.commons.lang3.tuple.Triple; 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.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -127,14 +128,14 @@ public void handleUpdateEvent(OMDBUpdateEvent event, * that are currently open in the backend. */ @Override - public Triple getTableSizeAndCount( - TableIterator> iterator) - throws IOException { + public Triple getTableSizeAndCount(String tableName, + OMMetadataManager omMetadataManager) throws IOException { long count = 0; long unReplicatedSize = 0; long replicatedSize = 0; - if (iterator != null) { + Table table = (Table) omMetadataManager.getTable(tableName); + try (TableIterator> iterator = table.iterator()) { while (iterator.hasNext()) { Table.KeyValue kv = iterator.next(); if (kv != null && kv.getValue() != null) { From 940b271241133649b0f4b719ea18384bf72d0e2a Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 4 Dec 2025 12:16:37 +0530 Subject: [PATCH 32/49] Removed unnecessary changes --- .../ozone/recon/ReconControllerModule.java | 10 +- .../impl/OzoneManagerServiceProviderImpl.java | 292 +++++++++--------- 2 files changed, 152 insertions(+), 150 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java index cea34593ed44..3f7e99056e44 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java @@ -139,12 +139,12 @@ static class ReconOmTaskBindingModule extends AbstractModule { protected void configure() { Multibinder taskBinder = Multibinder.newSetBinder(binder(), ReconOmTask.class); -// taskBinder.addBinding().to(ContainerKeyMapperTaskFSO.class); -// taskBinder.addBinding().to(ContainerKeyMapperTaskOBS.class); -// taskBinder.addBinding().to(FileSizeCountTaskFSO.class); -// taskBinder.addBinding().to(FileSizeCountTaskOBS.class); + taskBinder.addBinding().to(ContainerKeyMapperTaskFSO.class); + taskBinder.addBinding().to(ContainerKeyMapperTaskOBS.class); + taskBinder.addBinding().to(FileSizeCountTaskFSO.class); + taskBinder.addBinding().to(FileSizeCountTaskOBS.class); taskBinder.addBinding().to(OmTableInsightTask.class); -// taskBinder.addBinding().to(NSSummaryTask.class); + taskBinder.addBinding().to(NSSummaryTask.class); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 8b61b3d549ff..c548ac8b637f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -143,8 +143,6 @@ public class OzoneManagerServiceProviderImpl private ReconTaskStatusUpdaterManager taskStatusUpdaterManager; private TarExtractor tarExtractor; - private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); - /** * OM Snapshot related task names. */ @@ -365,7 +363,18 @@ private void startSyncDataFromOM(long initialDelay) { } private void stopSyncDataFromOMThread() { - scheduler.shutdownNow(); + scheduler.shutdown(); + try { + if (!scheduler.awaitTermination(30, TimeUnit.SECONDS)) { + scheduler.shutdownNow(); + if (!scheduler.awaitTermination(5, TimeUnit.SECONDS)) { + LOG.error("OM sync scheduler failed to terminate"); + } + } + } catch (InterruptedException e) { + scheduler.shutdownNow(); + Thread.currentThread().interrupt(); + } tarExtractor.stop(); LOG.debug("Shutdown the OM DB sync scheduler."); } @@ -704,152 +713,145 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = true; + boolean fullSnapshot = false; - if (reInitializeTasksCalled.compareAndSet(false, true)) { - LOG.info("Calling reprocess on Recon tasks."); - reconTaskController.reInitializeTasks(omMetadataManager, null); + if (currentSequenceNumber <= 0) { + fullSnapshot = true; } else { - LOG.info("reInitializeTasks already called once; skipping."); + // Get updates from OM and apply to local Recon OM DB and update task status in table + deltaReconTaskStatusUpdater.recordRunStart(); + int loopCount = 0; + long fromSequenceNumber = currentSequenceNumber; + long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; + /** + * This loop will continue to fetch and apply OM DB updates and with every + * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. + * It continues to fetch from OM till the lag, between OM DB WAL sequence number + * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. + * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. + */ + while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { + try (OMDBUpdatesHandler omdbUpdatesHandler = + new OMDBUpdatesHandler(omMetadataManager)) { + + // If interrupt was previously signalled, + // we should check for it before starting delta update sync. + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted during delta update."); + } + diffBetweenOMDbAndReconDBSeqNumber = + getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); + deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); + // Keeping last updated sequence number for both full and delta tasks to be same + // because sequence number of DB denotes and points to same OM DB copy of Recon, + // even though two different tasks are updating the DB at different conditions, but + // it tells the sync state with actual OM DB for the same Recon OM DB copy. + deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + deltaReconTaskStatusUpdater.recordRunCompletion(); + fullSnapshotReconTaskUpdater.updateDetails(); + // Update the current OM metadata manager in task controller + reconTaskController.updateOMMetadataManager(omMetadataManager); + + // Pass on DB update events to tasks that are listening. + reconTaskController.consumeOMEvents(new OMUpdateEventBatch( + omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); + + // Check if task reinitialization is needed due to buffer overflow or task failures + boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); + boolean tasksFailed = reconTaskController.hasTasksFailed(); + + if (bufferOverflowed || tasksFailed) { + ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? + ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : + ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; + + LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", + reason); + + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + + // Queue async reinitialization event - checkpoint creation and retry logic is handled internally + ReconTaskController.ReInitializationResult result = + reconTaskController.queueReInitializationEvent(reason); + + //TODO: Create a metric to track this event buffer overflow or task failure event + boolean triggerFullSnapshot = + Optional.ofNullable(result) + .map(r -> { + switch (r) { + case MAX_RETRIES_EXCEEDED: + LOG.warn( + "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + + "fallback"); + return true; + + case RETRY_LATER: + LOG.debug("Reinitialization event queueing will be retried in next iteration"); + return false; + + default: + LOG.info("Reinitialization event successfully queued"); + return false; + } + }) + .orElseGet(() -> { + LOG.error( + "ReInitializationResult is null, something went wrong in queueing reinitialization " + + "event"); + return true; + }); + + if (triggerFullSnapshot) { + fullSnapshot = true; + } + } + currentSequenceNumber = getCurrentOMDBSequenceNumber(); + LOG.debug("Updated current sequence number: {}", currentSequenceNumber); + loopCount++; + } catch (InterruptedException intEx) { + LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); + // We are updating the table even if it didn't run i.e. got interrupted beforehand + // to indicate that a task was supposed to run, but it didn't. + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + Thread.currentThread().interrupt(); + // Since thread is interrupted, we do not fall back to snapshot sync. + // Return with sync failed status. + return false; + } catch (Exception e) { + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", + e.getMessage()); + fullSnapshot = true; + } + if (fullSnapshot) { + break; + } + } + LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, + getCurrentOMDBSequenceNumber() - fromSequenceNumber); } -// if (currentSequenceNumber <= 0) { -// fullSnapshot = true; -// } else { -// // Get updates from OM and apply to local Recon OM DB and update task status in table -// deltaReconTaskStatusUpdater.recordRunStart(); -// int loopCount = 0; -// long fromSequenceNumber = currentSequenceNumber; -// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; -// /** -// * This loop will continue to fetch and apply OM DB updates and with every -// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. -// * It continues to fetch from OM till the lag, between OM DB WAL sequence number -// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. -// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. -// */ -// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { -// try (OMDBUpdatesHandler omdbUpdatesHandler = -// new OMDBUpdatesHandler(omMetadataManager)) { -// -// // If interrupt was previously signalled, -// // we should check for it before starting delta update sync. -// if (Thread.currentThread().isInterrupted()) { -// throw new InterruptedException("Thread interrupted during delta update."); -// } -// diffBetweenOMDbAndReconDBSeqNumber = -// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); -// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); -// // Keeping last updated sequence number for both full and delta tasks to be same -// // because sequence number of DB denotes and points to same OM DB copy of Recon, -// // even though two different tasks are updating the DB at different conditions, but -// // it tells the sync state with actual OM DB for the same Recon OM DB copy. -// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// deltaReconTaskStatusUpdater.recordRunCompletion(); -// fullSnapshotReconTaskUpdater.updateDetails(); -// // Update the current OM metadata manager in task controller -// reconTaskController.updateOMMetadataManager(omMetadataManager); -// -// // Pass on DB update events to tasks that are listening. -// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( -// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); -// -// // Check if task reinitialization is needed due to buffer overflow or task failures -// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); -// boolean tasksFailed = reconTaskController.hasTasksFailed(); -// -// if (bufferOverflowed || tasksFailed) { -// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? -// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : -// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; -// -// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", -// reason); -// -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// -// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally -// ReconTaskController.ReInitializationResult result = -// reconTaskController.queueReInitializationEvent(reason); -// -// //TODO: Create a metric to track this event buffer overflow or task failure event -// boolean triggerFullSnapshot = -// Optional.ofNullable(result) -// .map(r -> { -// switch (r) { -// case MAX_RETRIES_EXCEEDED: -// LOG.warn( -// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + -// "fallback"); -// return true; -// -// case RETRY_LATER: -// LOG.debug("Reinitialization event queueing will be retried in next iteration"); -// return false; -// -// default: -// LOG.info("Reinitialization event successfully queued"); -// return false; -// } -// }) -// .orElseGet(() -> { -// LOG.error( -// "ReInitializationResult is null, something went wrong in queueing reinitialization " + -// "event"); -// return true; -// }); -// -// if (triggerFullSnapshot) { -// fullSnapshot = true; -// } -// } -// currentSequenceNumber = getCurrentOMDBSequenceNumber(); -// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); -// loopCount++; -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); -// // We are updating the table even if it didn't run i.e. got interrupted beforehand -// // to indicate that a task was supposed to run, but it didn't. -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// Thread.currentThread().interrupt(); -// // Since thread is interrupted, we do not fall back to snapshot sync. -// // Return with sync failed status. -// return false; -// } catch (Exception e) { -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", -// e.getMessage()); -// fullSnapshot = true; -// } -// if (fullSnapshot) { -// break; -// } -// } -// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, -// getCurrentOMDBSequenceNumber() - fromSequenceNumber); -// } - -// if (fullSnapshot) { -// try { -// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Snapshot update sync thread was interrupted."); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// Thread.currentThread().interrupt(); -// // Mark sync status as failed. -// return false; -// } catch (Exception e) { -// metrics.incrNumSnapshotRequestsFailed(); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); -// // Update health status in ReconContext -// reconContext.updateHealthStatus(new AtomicBoolean(false)); -// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); -// } -// } + if (fullSnapshot) { + try { + executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); + } catch (InterruptedException intEx) { + LOG.error("OM DB Snapshot update sync thread was interrupted."); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + Thread.currentThread().interrupt(); + // Mark sync status as failed. + return false; + } catch (Exception e) { + metrics.incrNumSnapshotRequestsFailed(); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + LOG.error("Unable to update Recon's metadata with new OM DB. ", e); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); + } + } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); From 702e92bf9c88e655e0a3162534a60a01291be1ea Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 4 Dec 2025 12:52:55 +0530 Subject: [PATCH 33/49] Removed some extra changes --- .../spi/impl/OzoneManagerServiceProviderImpl.java | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index c548ac8b637f..0119bf373b5a 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -118,7 +118,7 @@ public class OzoneManagerServiceProviderImpl private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; - + private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); private File omSnapshotDBParentDir = null; private File reconDbDir = null; private String omDBSnapshotUrl; @@ -364,17 +364,6 @@ private void startSyncDataFromOM(long initialDelay) { private void stopSyncDataFromOMThread() { scheduler.shutdown(); - try { - if (!scheduler.awaitTermination(30, TimeUnit.SECONDS)) { - scheduler.shutdownNow(); - if (!scheduler.awaitTermination(5, TimeUnit.SECONDS)) { - LOG.error("OM sync scheduler failed to terminate"); - } - } - } catch (InterruptedException e) { - scheduler.shutdownNow(); - Thread.currentThread().interrupt(); - } tarExtractor.stop(); LOG.debug("Shutdown the OM DB sync scheduler."); } From 58bcccdb5c90eac8f56ece7edda64ebcaa6d1a07 Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 4 Dec 2025 12:53:33 +0530 Subject: [PATCH 34/49] Reverted back to the old change --- .../ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 0119bf373b5a..11824f907b44 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -363,7 +363,7 @@ private void startSyncDataFromOM(long initialDelay) { } private void stopSyncDataFromOMThread() { - scheduler.shutdown(); + scheduler.shutdownNow(); tarExtractor.stop(); LOG.debug("Shutdown the OM DB sync scheduler."); } From f55fe25f015927020fc2c76f7833d6bc114fe68e Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 4 Dec 2025 14:57:52 +0530 Subject: [PATCH 35/49] Fixed some more code --- .../impl/OzoneManagerServiceProviderImpl.java | 278 +++++++++--------- .../tasks/ContainerKeyMapperTaskFSO.java | 2 +- pom.xml | 2 +- 3 files changed, 144 insertions(+), 138 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 11824f907b44..a8a5e1b8751c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -702,145 +702,151 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = false; + boolean fullSnapshot = true; - if (currentSequenceNumber <= 0) { - fullSnapshot = true; + if (reInitializeTasksCalled.compareAndSet(false, true)) { + LOG.info("Calling reprocess on Recon tasks."); + reconTaskController.reInitializeTasks(omMetadataManager, null); } else { - // Get updates from OM and apply to local Recon OM DB and update task status in table - deltaReconTaskStatusUpdater.recordRunStart(); - int loopCount = 0; - long fromSequenceNumber = currentSequenceNumber; - long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; - /** - * This loop will continue to fetch and apply OM DB updates and with every - * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. - * It continues to fetch from OM till the lag, between OM DB WAL sequence number - * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. - * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. - */ - while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { - try (OMDBUpdatesHandler omdbUpdatesHandler = - new OMDBUpdatesHandler(omMetadataManager)) { - - // If interrupt was previously signalled, - // we should check for it before starting delta update sync. - if (Thread.currentThread().isInterrupted()) { - throw new InterruptedException("Thread interrupted during delta update."); - } - diffBetweenOMDbAndReconDBSeqNumber = - getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); - deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); - // Keeping last updated sequence number for both full and delta tasks to be same - // because sequence number of DB denotes and points to same OM DB copy of Recon, - // even though two different tasks are updating the DB at different conditions, but - // it tells the sync state with actual OM DB for the same Recon OM DB copy. - deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - deltaReconTaskStatusUpdater.recordRunCompletion(); - fullSnapshotReconTaskUpdater.updateDetails(); - // Update the current OM metadata manager in task controller - reconTaskController.updateOMMetadataManager(omMetadataManager); - - // Pass on DB update events to tasks that are listening. - reconTaskController.consumeOMEvents(new OMUpdateEventBatch( - omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); - - // Check if task reinitialization is needed due to buffer overflow or task failures - boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); - boolean tasksFailed = reconTaskController.hasTasksFailed(); - - if (bufferOverflowed || tasksFailed) { - ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? - ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : - ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; - - LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", - reason); - - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - - // Queue async reinitialization event - checkpoint creation and retry logic is handled internally - ReconTaskController.ReInitializationResult result = - reconTaskController.queueReInitializationEvent(reason); - - //TODO: Create a metric to track this event buffer overflow or task failure event - boolean triggerFullSnapshot = - Optional.ofNullable(result) - .map(r -> { - switch (r) { - case MAX_RETRIES_EXCEEDED: - LOG.warn( - "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + - "fallback"); - return true; - - case RETRY_LATER: - LOG.debug("Reinitialization event queueing will be retried in next iteration"); - return false; - - default: - LOG.info("Reinitialization event successfully queued"); - return false; - } - }) - .orElseGet(() -> { - LOG.error( - "ReInitializationResult is null, something went wrong in queueing reinitialization " + - "event"); - return true; - }); - - if (triggerFullSnapshot) { - fullSnapshot = true; - } - } - currentSequenceNumber = getCurrentOMDBSequenceNumber(); - LOG.debug("Updated current sequence number: {}", currentSequenceNumber); - loopCount++; - } catch (InterruptedException intEx) { - LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); - // We are updating the table even if it didn't run i.e. got interrupted beforehand - // to indicate that a task was supposed to run, but it didn't. - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - Thread.currentThread().interrupt(); - // Since thread is interrupted, we do not fall back to snapshot sync. - // Return with sync failed status. - return false; - } catch (Exception e) { - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", - e.getMessage()); - fullSnapshot = true; - } - if (fullSnapshot) { - break; - } - } - LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, - getCurrentOMDBSequenceNumber() - fromSequenceNumber); - } - - if (fullSnapshot) { - try { - executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); - } catch (InterruptedException intEx) { - LOG.error("OM DB Snapshot update sync thread was interrupted."); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - Thread.currentThread().interrupt(); - // Mark sync status as failed. - return false; - } catch (Exception e) { - metrics.incrNumSnapshotRequestsFailed(); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - LOG.error("Unable to update Recon's metadata with new OM DB. ", e); - // Update health status in ReconContext - reconContext.updateHealthStatus(new AtomicBoolean(false)); - reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); - } + LOG.info("reInitializeTasks already called once; skipping."); } +// if (currentSequenceNumber <= 0) { +// fullSnapshot = true; +// } else { +// // Get updates from OM and apply to local Recon OM DB and update task status in table +// deltaReconTaskStatusUpdater.recordRunStart(); +// int loopCount = 0; +// long fromSequenceNumber = currentSequenceNumber; +// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; +// /** +// * This loop will continue to fetch and apply OM DB updates and with every +// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. +// * It continues to fetch from OM till the lag, between OM DB WAL sequence number +// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. +// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. +// */ +// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { +// try (OMDBUpdatesHandler omdbUpdatesHandler = +// new OMDBUpdatesHandler(omMetadataManager)) { +// +// // If interrupt was previously signalled, +// // we should check for it before starting delta update sync. +// if (Thread.currentThread().isInterrupted()) { +// throw new InterruptedException("Thread interrupted during delta update."); +// } +// diffBetweenOMDbAndReconDBSeqNumber = +// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); +// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); +// // Keeping last updated sequence number for both full and delta tasks to be same +// // because sequence number of DB denotes and points to same OM DB copy of Recon, +// // even though two different tasks are updating the DB at different conditions, but +// // it tells the sync state with actual OM DB for the same Recon OM DB copy. +// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// deltaReconTaskStatusUpdater.recordRunCompletion(); +// fullSnapshotReconTaskUpdater.updateDetails(); +// // Update the current OM metadata manager in task controller +// reconTaskController.updateOMMetadataManager(omMetadataManager); +// +// // Pass on DB update events to tasks that are listening. +// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( +// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); +// +// // Check if task reinitialization is needed due to buffer overflow or task failures +// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); +// boolean tasksFailed = reconTaskController.hasTasksFailed(); +// +// if (bufferOverflowed || tasksFailed) { +// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? +// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : +// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; +// +// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", +// reason); +// +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// +// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally +// ReconTaskController.ReInitializationResult result = +// reconTaskController.queueReInitializationEvent(reason); +// +// //TODO: Create a metric to track this event buffer overflow or task failure event +// boolean triggerFullSnapshot = +// Optional.ofNullable(result) +// .map(r -> { +// switch (r) { +// case MAX_RETRIES_EXCEEDED: +// LOG.warn( +// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + +// "fallback"); +// return true; +// +// case RETRY_LATER: +// LOG.debug("Reinitialization event queueing will be retried in next iteration"); +// return false; +// +// default: +// LOG.info("Reinitialization event successfully queued"); +// return false; +// } +// }) +// .orElseGet(() -> { +// LOG.error( +// "ReInitializationResult is null, something went wrong in queueing reinitialization " + +// "event"); +// return true; +// }); +// +// if (triggerFullSnapshot) { +// fullSnapshot = true; +// } +// } +// currentSequenceNumber = getCurrentOMDBSequenceNumber(); +// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); +// loopCount++; +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); +// // We are updating the table even if it didn't run i.e. got interrupted beforehand +// // to indicate that a task was supposed to run, but it didn't. +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// Thread.currentThread().interrupt(); +// // Since thread is interrupted, we do not fall back to snapshot sync. +// // Return with sync failed status. +// return false; +// } catch (Exception e) { +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", +// e.getMessage()); +// fullSnapshot = true; +// } +// if (fullSnapshot) { +// break; +// } +// } +// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, +// getCurrentOMDBSequenceNumber() - fromSequenceNumber); +// } + +// if (fullSnapshot) { +// try { +// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Snapshot update sync thread was interrupted."); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// Thread.currentThread().interrupt(); +// // Mark sync status as failed. +// return false; +// } catch (Exception e) { +// metrics.incrNumSnapshotRequestsFailed(); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); +// // Update health status in ReconContext +// reconContext.updateHealthStatus(new AtomicBoolean(false)); +// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); +// } +// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java index 59f5bdd1efba..9b3374248bd3 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTaskFSO.java @@ -66,7 +66,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { ReconServerConfigKeys.OZONE_RECON_TASK_REPROCESS_MAX_WORKERS_DEFAULT); boolean result = ContainerKeyMapperHelper.reprocess( omMetadataManager, reconContainerMetadataManager, - BucketLayout.FILE_SYSTEM_OPTIMIZED, getTaskName(), 200000, + BucketLayout.FILE_SYSTEM_OPTIMIZED, getTaskName(), containerKeyFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); return buildTaskResult(result); } diff --git a/pom.xml b/pom.xml index f1e5bbf412df..db7fa29ddc02 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ 1.9.7 3.27.6 1.12.788 - 2.36.2 + 2.37.3 0.8.0.RELEASE 1.82 3.6.1 From 47eaf82eeb0f7f4d61f17d5f7e125ee631dfef3c Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 4 Dec 2025 15:09:54 +0530 Subject: [PATCH 36/49] Removed testing code --- .../impl/OzoneManagerServiceProviderImpl.java | 280 +++++++++--------- 1 file changed, 137 insertions(+), 143 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index a8a5e1b8751c..874049bfa89e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -118,7 +118,7 @@ public class OzoneManagerServiceProviderImpl private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; - private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); + private File omSnapshotDBParentDir = null; private File reconDbDir = null; private String omDBSnapshotUrl; @@ -702,151 +702,145 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = true; + boolean fullSnapshot = false; - if (reInitializeTasksCalled.compareAndSet(false, true)) { - LOG.info("Calling reprocess on Recon tasks."); - reconTaskController.reInitializeTasks(omMetadataManager, null); + if (currentSequenceNumber <= 0) { + fullSnapshot = true; } else { - LOG.info("reInitializeTasks already called once; skipping."); + // Get updates from OM and apply to local Recon OM DB and update task status in table + deltaReconTaskStatusUpdater.recordRunStart(); + int loopCount = 0; + long fromSequenceNumber = currentSequenceNumber; + long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; + /** + * This loop will continue to fetch and apply OM DB updates and with every + * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. + * It continues to fetch from OM till the lag, between OM DB WAL sequence number + * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. + * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. + */ + while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { + try (OMDBUpdatesHandler omdbUpdatesHandler = + new OMDBUpdatesHandler(omMetadataManager)) { + + // If interrupt was previously signalled, + // we should check for it before starting delta update sync. + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted during delta update."); + } + diffBetweenOMDbAndReconDBSeqNumber = + getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); + deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); + // Keeping last updated sequence number for both full and delta tasks to be same + // because sequence number of DB denotes and points to same OM DB copy of Recon, + // even though two different tasks are updating the DB at different conditions, but + // it tells the sync state with actual OM DB for the same Recon OM DB copy. + deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + deltaReconTaskStatusUpdater.recordRunCompletion(); + fullSnapshotReconTaskUpdater.updateDetails(); + // Update the current OM metadata manager in task controller + reconTaskController.updateOMMetadataManager(omMetadataManager); + + // Pass on DB update events to tasks that are listening. + reconTaskController.consumeOMEvents(new OMUpdateEventBatch( + omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); + + // Check if task reinitialization is needed due to buffer overflow or task failures + boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); + boolean tasksFailed = reconTaskController.hasTasksFailed(); + + if (bufferOverflowed || tasksFailed) { + ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? + ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : + ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; + + LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", + reason); + + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + + // Queue async reinitialization event - checkpoint creation and retry logic is handled internally + ReconTaskController.ReInitializationResult result = + reconTaskController.queueReInitializationEvent(reason); + + //TODO: Create a metric to track this event buffer overflow or task failure event + boolean triggerFullSnapshot = + Optional.ofNullable(result) + .map(r -> { + switch (r) { + case MAX_RETRIES_EXCEEDED: + LOG.warn( + "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + + "fallback"); + return true; + + case RETRY_LATER: + LOG.debug("Reinitialization event queueing will be retried in next iteration"); + return false; + + default: + LOG.info("Reinitialization event successfully queued"); + return false; + } + }) + .orElseGet(() -> { + LOG.error( + "ReInitializationResult is null, something went wrong in queueing reinitialization " + + "event"); + return true; + }); + + if (triggerFullSnapshot) { + fullSnapshot = true; + } + } + currentSequenceNumber = getCurrentOMDBSequenceNumber(); + LOG.debug("Updated current sequence number: {}", currentSequenceNumber); + loopCount++; + } catch (InterruptedException intEx) { + LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); + // We are updating the table even if it didn't run i.e. got interrupted beforehand + // to indicate that a task was supposed to run, but it didn't. + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + Thread.currentThread().interrupt(); + // Since thread is interrupted, we do not fall back to snapshot sync. + // Return with sync failed status. + return false; + } catch (Exception e) { + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", + e.getMessage()); + fullSnapshot = true; + } + if (fullSnapshot) { + break; + } + } + LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, + getCurrentOMDBSequenceNumber() - fromSequenceNumber); + } + + if (fullSnapshot) { + try { + executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); + } catch (InterruptedException intEx) { + LOG.error("OM DB Snapshot update sync thread was interrupted."); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + Thread.currentThread().interrupt(); + // Mark sync status as failed. + return false; + } catch (Exception e) { + metrics.incrNumSnapshotRequestsFailed(); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + LOG.error("Unable to update Recon's metadata with new OM DB. ", e); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); + } } -// if (currentSequenceNumber <= 0) { -// fullSnapshot = true; -// } else { -// // Get updates from OM and apply to local Recon OM DB and update task status in table -// deltaReconTaskStatusUpdater.recordRunStart(); -// int loopCount = 0; -// long fromSequenceNumber = currentSequenceNumber; -// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; -// /** -// * This loop will continue to fetch and apply OM DB updates and with every -// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. -// * It continues to fetch from OM till the lag, between OM DB WAL sequence number -// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. -// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. -// */ -// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { -// try (OMDBUpdatesHandler omdbUpdatesHandler = -// new OMDBUpdatesHandler(omMetadataManager)) { -// -// // If interrupt was previously signalled, -// // we should check for it before starting delta update sync. -// if (Thread.currentThread().isInterrupted()) { -// throw new InterruptedException("Thread interrupted during delta update."); -// } -// diffBetweenOMDbAndReconDBSeqNumber = -// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); -// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); -// // Keeping last updated sequence number for both full and delta tasks to be same -// // because sequence number of DB denotes and points to same OM DB copy of Recon, -// // even though two different tasks are updating the DB at different conditions, but -// // it tells the sync state with actual OM DB for the same Recon OM DB copy. -// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// deltaReconTaskStatusUpdater.recordRunCompletion(); -// fullSnapshotReconTaskUpdater.updateDetails(); -// // Update the current OM metadata manager in task controller -// reconTaskController.updateOMMetadataManager(omMetadataManager); -// -// // Pass on DB update events to tasks that are listening. -// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( -// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); -// -// // Check if task reinitialization is needed due to buffer overflow or task failures -// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); -// boolean tasksFailed = reconTaskController.hasTasksFailed(); -// -// if (bufferOverflowed || tasksFailed) { -// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? -// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : -// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; -// -// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", -// reason); -// -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// -// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally -// ReconTaskController.ReInitializationResult result = -// reconTaskController.queueReInitializationEvent(reason); -// -// //TODO: Create a metric to track this event buffer overflow or task failure event -// boolean triggerFullSnapshot = -// Optional.ofNullable(result) -// .map(r -> { -// switch (r) { -// case MAX_RETRIES_EXCEEDED: -// LOG.warn( -// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + -// "fallback"); -// return true; -// -// case RETRY_LATER: -// LOG.debug("Reinitialization event queueing will be retried in next iteration"); -// return false; -// -// default: -// LOG.info("Reinitialization event successfully queued"); -// return false; -// } -// }) -// .orElseGet(() -> { -// LOG.error( -// "ReInitializationResult is null, something went wrong in queueing reinitialization " + -// "event"); -// return true; -// }); -// -// if (triggerFullSnapshot) { -// fullSnapshot = true; -// } -// } -// currentSequenceNumber = getCurrentOMDBSequenceNumber(); -// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); -// loopCount++; -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); -// // We are updating the table even if it didn't run i.e. got interrupted beforehand -// // to indicate that a task was supposed to run, but it didn't. -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// Thread.currentThread().interrupt(); -// // Since thread is interrupted, we do not fall back to snapshot sync. -// // Return with sync failed status. -// return false; -// } catch (Exception e) { -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", -// e.getMessage()); -// fullSnapshot = true; -// } -// if (fullSnapshot) { -// break; -// } -// } -// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, -// getCurrentOMDBSequenceNumber() - fromSequenceNumber); -// } - -// if (fullSnapshot) { -// try { -// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Snapshot update sync thread was interrupted."); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// Thread.currentThread().interrupt(); -// // Mark sync status as failed. -// return false; -// } catch (Exception e) { -// metrics.incrNumSnapshotRequestsFailed(); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); -// // Update health status in ReconContext -// reconContext.updateHealthStatus(new AtomicBoolean(false)); -// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); -// } -// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); From 980135d8de3eeae5d6bfcc4d5676ec6986157483 Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 4 Dec 2025 16:15:56 +0530 Subject: [PATCH 37/49] Recon: Improve type safety in OmTableHandlers and fix ACTIVE_TASK_COUNT leak on initialization exception. --- .../ozone/recon/tasks/ContainerKeyMapperHelper.java | 7 +++---- .../ozone/recon/tasks/DeletedKeysInsightHandler.java | 11 ++++++----- .../ozone/recon/tasks/OpenKeysInsightHandler.java | 9 +++++---- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index de8b6628fda7..756d4ec287f8 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -96,6 +96,7 @@ private static void initializeContainerKeyMapperIfNeeded( String taskName) { synchronized (INITIALIZATION_LOCK) { + ACTIVE_TASK_COUNT.incrementAndGet(); // Check if already initialized by another task if (ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.compareAndSet(false, true)) { try { @@ -104,12 +105,10 @@ private static void initializeContainerKeyMapperIfNeeded( // Step 2: Clear shared map SHARED_CONTAINER_KEY_COUNT_MAP.clear(); - - // Step 3: Increment the refrence counter for active tasks - ACTIVE_TASK_COUNT.incrementAndGet(); } catch (Exception e) { - // CRITICAL: Reset flag so another task can retry + // CRITICAL: Decrement counter and reset flag so another task can retry + ACTIVE_TASK_COUNT.decrementAndGet(); ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); LOG.error("{}: Container Key Mapper initialization failed. Resetting flag for retry.", taskName, e); throw new RuntimeException("Container Key Mapper initialization failed", e); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java index 43386c556f9e..1339325ffc12 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java @@ -119,13 +119,14 @@ public Triple getTableSizeAndCount(String tableName, long unReplicatedSize = 0; long replicatedSize = 0; - Table table = (Table) omMetadataManager.getTable(tableName); - try (TableIterator> iterator = table.iterator()) { + @SuppressWarnings("unchecked") + Table table = + (Table) omMetadataManager.getTable(tableName); + try (TableIterator> iterator = table.iterator()) { while (iterator.hasNext()) { - Table.KeyValue kv = iterator.next(); + Table.KeyValue kv = iterator.next(); if (kv != null && kv.getValue() != null) { - RepeatedOmKeyInfo repeatedOmKeyInfo = (RepeatedOmKeyInfo) kv - .getValue(); + RepeatedOmKeyInfo repeatedOmKeyInfo = kv.getValue(); Pair result = repeatedOmKeyInfo.getTotalSize(); unReplicatedSize += result.getRight(); replicatedSize += result.getLeft(); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java index 6264ad3b2b4a..c47f423f4ab4 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java @@ -134,12 +134,13 @@ public Triple getTableSizeAndCount(String tableName, long unReplicatedSize = 0; long replicatedSize = 0; - Table table = (Table) omMetadataManager.getTable(tableName); - try (TableIterator> iterator = table.iterator()) { + @SuppressWarnings("unchecked") + Table table = (Table) omMetadataManager.getTable(tableName); + try (TableIterator> iterator = table.iterator()) { while (iterator.hasNext()) { - Table.KeyValue kv = iterator.next(); + Table.KeyValue kv = iterator.next(); if (kv != null && kv.getValue() != null) { - OmKeyInfo omKeyInfo = (OmKeyInfo) kv.getValue(); + OmKeyInfo omKeyInfo = kv.getValue(); unReplicatedSize += omKeyInfo.getDataSize(); replicatedSize += omKeyInfo.getReplicatedSize(); count++; From cff7278639f028f651febb3656b77f196f4579e5 Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 8 Dec 2025 19:59:21 +0530 Subject: [PATCH 38/49] Removed the supress warning --- .../hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java | 4 +--- .../hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java | 1 - 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java index 1339325ffc12..54e375ae0d6f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/DeletedKeysInsightHandler.java @@ -119,9 +119,7 @@ public Triple getTableSizeAndCount(String tableName, long unReplicatedSize = 0; long replicatedSize = 0; - @SuppressWarnings("unchecked") - Table table = - (Table) omMetadataManager.getTable(tableName); + Table table = omMetadataManager.getDeletedTable(); try (TableIterator> iterator = table.iterator()) { while (iterator.hasNext()) { Table.KeyValue kv = iterator.next(); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java index c47f423f4ab4..b78e8cb1518f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OpenKeysInsightHandler.java @@ -134,7 +134,6 @@ public Triple getTableSizeAndCount(String tableName, long unReplicatedSize = 0; long replicatedSize = 0; - @SuppressWarnings("unchecked") Table table = (Table) omMetadataManager.getTable(tableName); try (TableIterator> iterator = table.iterator()) { while (iterator.hasNext()) { From 04df20763cda668b36e14af9b46ec68b71c3a130 Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 9 Dec 2025 11:29:33 +0530 Subject: [PATCH 39/49] Fixed checkstyle --- .../recon/tasks/ContainerKeyMapperHelper.java | 16 +++--- .../recon/tasks/FileSizeCountTaskHelper.java | 56 ++++++++++--------- .../util/ParallelTableIteratorOperation.java | 3 +- 3 files changed, 39 insertions(+), 36 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 756d4ec287f8..266beee7af3f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -17,6 +17,7 @@ package org.apache.hadoop.ozone.recon.tasks; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.io.UncheckedIOException; import java.time.Duration; @@ -33,7 +34,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; -import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.hdds.utils.db.StringCodec; import org.apache.hadoop.hdds.utils.db.Table; @@ -119,6 +119,7 @@ private static void initializeContainerKeyMapperIfNeeded( } } + @SuppressWarnings("checkstyle:ParameterNumber") public static boolean reprocess(OMMetadataManager omMetadataManager, ReconContainerMetadataManager reconContainerMetadataManager, BucketLayout bucketLayout, @@ -127,7 +128,6 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, int maxIterators, int maxWorkers, int maxKeysInMemory) { - try { LOG.info("{}: Starting reprocess for bucket layout {}", taskName, bucketLayout); Instant start = Instant.now(); @@ -138,7 +138,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); // Divide threshold by worker count so each worker flushes independently - final long PER_WORKER_THRESHOLD = Math.max(1, containerKeyFlushToDBMaxThreshold / maxWorkers); + final long perWorkerThreshold = Math.max(1, containerKeyFlushToDBMaxThreshold / maxWorkers); // Map thread IDs to worker-specific local maps for lockless updates Map> allLocalMaps = new ConcurrentHashMap<>(); @@ -153,7 +153,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, reconContainerMetadataManager); // Flush this worker's map when it reaches threshold - if (containerKeyPrefixMap.size() >= PER_WORKER_THRESHOLD) { + if (containerKeyPrefixMap.size() >= perWorkerThreshold) { if (!flushAndCommitContainerKeyInfoToDB(containerKeyPrefixMap, Collections.emptyMap(), reconContainerMetadataManager)) { throw new UncheckedIOException(new IOException("Unable to flush containerKey information to the DB")); @@ -167,7 +167,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, try (ParallelTableIteratorOperation keyIter = new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, PER_WORKER_THRESHOLD)) { + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, perWorkerThreshold)) { keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } @@ -288,7 +288,8 @@ public static boolean process(OMUpdateEventBatch events, // Convert local Long map to AtomicLong map for writeToTheDB compatibility Map localContainerKeyCountMapAtomic = new ConcurrentHashMap<>(); localContainerKeyCountMap.forEach((k, v) -> localContainerKeyCountMapAtomic.put(k, new AtomicLong(v))); - writeToTheDB(localContainerKeyMap, localContainerKeyCountMapAtomic, deletedKeyCountList, reconContainerMetadataManager); + writeToTheDB(localContainerKeyMap, localContainerKeyCountMapAtomic, deletedKeyCountList, + reconContainerMetadataManager); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); return false; @@ -514,7 +515,8 @@ public static boolean flushAndCommitContainerKeyInfoToDB( try { // No deleted container list needed since "reprocess" only has put operations - writeToTheDB(localContainerKeyMap, sharedContainerKeyCountMap, Collections.emptyList(), reconContainerMetadataManager); + writeToTheDB(localContainerKeyMap, sharedContainerKeyCountMap, Collections.emptyList(), + reconContainerMetadataManager); // Only clear localContainerKeyMap (per-task), keep sharedContainerKeyCountMap for other tasks localContainerKeyMap.clear(); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java index dd3a974198b7..b82fcd556a1b 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskHelper.java @@ -102,6 +102,7 @@ public static void truncateFileCountTableIfNeeded(ReconFileMetadataManager recon /** * Executes the reprocess method using RocksDB for the given task. */ + @SuppressWarnings("checkstyle:ParameterNumber") public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManager, ReconFileMetadataManager reconFileMetadataManager, BucketLayout bucketLayout, @@ -139,6 +140,7 @@ public static ReconOmTask.TaskResult reprocess(OMMetadataManager omMetadataManag * Iterates over the OM DB keys for the given bucket layout using lockless per-worker maps. * Each worker maintains its own map to eliminate read lock contention. */ + @SuppressWarnings("checkstyle:ParameterNumber") public static boolean reprocessBucketLayout(BucketLayout bucketLayout, OMMetadataManager omMetadataManager, Map fileSizeCountMap, @@ -151,48 +153,48 @@ public static boolean reprocessBucketLayout(BucketLayout bucketLayout, Table omKeyInfoTable = omMetadataManager.getKeyTable(bucketLayout); // Divide threshold by worker count so each worker flushes independently - final long PER_WORKER_THRESHOLD = Math.max(1, fileSizeCountFlushThreshold / maxWorkers); - + final long perWorkerThreshold = Math.max(1, fileSizeCountFlushThreshold / maxWorkers); + // Map thread IDs to worker-specific maps for lockless updates Map> allMap = new ConcurrentHashMap<>(); - + // Lock for coordinating DB flush operations only Object flushLock = new Object(); - + // Lambda executed by workers for each key Function, Void> kvOperation = kv -> { - // Get or create this worker's private map using thread ID - Map workerFileSizeCountMap = allMap.computeIfAbsent( - Thread.currentThread().getId(), k -> new HashMap<>()); - - // Update worker's private map without locks - handlePutKeyEvent(kv.getValue(), workerFileSizeCountMap); - - // Flush this worker's map when it reaches threshold - if (workerFileSizeCountMap.size() >= PER_WORKER_THRESHOLD) { - synchronized (flushLock) { - writeCountsToDB(workerFileSizeCountMap, reconFileMetadataManager); - workerFileSizeCountMap.clear(); - } + // Get or create this worker's private map using thread ID + Map workerFileSizeCountMap = allMap.computeIfAbsent( + Thread.currentThread().getId(), k -> new HashMap<>()); + + // Update worker's private map without locks + handlePutKeyEvent(kv.getValue(), workerFileSizeCountMap); + + // Flush this worker's map when it reaches threshold + if (workerFileSizeCountMap.size() >= perWorkerThreshold) { + synchronized (flushLock) { + writeCountsToDB(workerFileSizeCountMap, reconFileMetadataManager); + workerFileSizeCountMap.clear(); } - return null; + } + return null; }; - + try (ParallelTableIteratorOperation keyIter = new ParallelTableIteratorOperation<>(omMetadataManager, omKeyInfoTable, - StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, PER_WORKER_THRESHOLD)) { - keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); + StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, perWorkerThreshold)) { + keyIter.performTaskOnTableVals(taskName, null, null, kvOperation); } catch (Exception ex) { - LOG.error("Unable to populate File Size Count for {} in RocksDB.", taskName, ex); - return false; + LOG.error("Unable to populate File Size Count for {} in RocksDB.", taskName, ex); + return false; } // Final flush: Write remaining entries from all worker maps to DB for (Map workerFileSizeCountMap : allMap.values()) { - if (!workerFileSizeCountMap.isEmpty()) { - writeCountsToDB(workerFileSizeCountMap, reconFileMetadataManager); - workerFileSizeCountMap.clear(); - } + if (!workerFileSizeCountMap.isEmpty()) { + writeCountsToDB(workerFileSizeCountMap, reconFileMetadataManager); + workerFileSizeCountMap.clear(); + } } return true; diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java index bee48af7ade6..627fee974b2a 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -58,7 +58,6 @@ public class ParallelTableIteratorOperation, V> implemen private final ExecutorService iteratorExecutor; // 5 private final ExecutorService valueExecutors; // 20 - private final int maxNumberOfVals; private final OMMetadataManager metadataManager; private final int maxIteratorTasks; @@ -66,6 +65,7 @@ public class ParallelTableIteratorOperation, V> implemen private final long logCountThreshold; private static final Logger LOG = LoggerFactory.getLogger(ParallelTableIteratorOperation.class); + public ParallelTableIteratorOperation(OMMetadataManager metadataManager, Table table, Codec keyCodec, int iteratorCount, int workerCount, int maxNumberOfValsInMemory, long logThreshold) { @@ -89,7 +89,6 @@ public ParallelTableIteratorOperation(OMMetadataManager metadataManager, Table getBounds(K startKey, K endKey) throws IOException { Set keys = new HashSet<>(); From bed2151095bec7e502286f66bf973861d37d64bb Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 9 Dec 2025 12:11:04 +0530 Subject: [PATCH 40/49] Fixed findbugs --- .../impl/OzoneManagerServiceProviderImpl.java | 281 +++++++++--------- .../recon/tasks/ContainerKeyMapperHelper.java | 2 +- .../util/ParallelTableIteratorOperation.java | 5 +- 3 files changed, 148 insertions(+), 140 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 874049bfa89e..c5c4a688327a 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -118,7 +118,7 @@ public class OzoneManagerServiceProviderImpl private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; - + private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); private File omSnapshotDBParentDir = null; private File reconDbDir = null; private String omDBSnapshotUrl; @@ -702,145 +702,152 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = false; + boolean fullSnapshot = true; - if (currentSequenceNumber <= 0) { - fullSnapshot = true; - } else { - // Get updates from OM and apply to local Recon OM DB and update task status in table - deltaReconTaskStatusUpdater.recordRunStart(); - int loopCount = 0; - long fromSequenceNumber = currentSequenceNumber; - long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; - /** - * This loop will continue to fetch and apply OM DB updates and with every - * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. - * It continues to fetch from OM till the lag, between OM DB WAL sequence number - * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. - * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. - */ - while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { - try (OMDBUpdatesHandler omdbUpdatesHandler = - new OMDBUpdatesHandler(omMetadataManager)) { - - // If interrupt was previously signalled, - // we should check for it before starting delta update sync. - if (Thread.currentThread().isInterrupted()) { - throw new InterruptedException("Thread interrupted during delta update."); - } - diffBetweenOMDbAndReconDBSeqNumber = - getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); - deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); - // Keeping last updated sequence number for both full and delta tasks to be same - // because sequence number of DB denotes and points to same OM DB copy of Recon, - // even though two different tasks are updating the DB at different conditions, but - // it tells the sync state with actual OM DB for the same Recon OM DB copy. - deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); - deltaReconTaskStatusUpdater.recordRunCompletion(); - fullSnapshotReconTaskUpdater.updateDetails(); - // Update the current OM metadata manager in task controller - reconTaskController.updateOMMetadataManager(omMetadataManager); - - // Pass on DB update events to tasks that are listening. - reconTaskController.consumeOMEvents(new OMUpdateEventBatch( - omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); - - // Check if task reinitialization is needed due to buffer overflow or task failures - boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); - boolean tasksFailed = reconTaskController.hasTasksFailed(); - - if (bufferOverflowed || tasksFailed) { - ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? - ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : - ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; - - LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", - reason); - - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - - // Queue async reinitialization event - checkpoint creation and retry logic is handled internally - ReconTaskController.ReInitializationResult result = - reconTaskController.queueReInitializationEvent(reason); - - //TODO: Create a metric to track this event buffer overflow or task failure event - boolean triggerFullSnapshot = - Optional.ofNullable(result) - .map(r -> { - switch (r) { - case MAX_RETRIES_EXCEEDED: - LOG.warn( - "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + - "fallback"); - return true; - - case RETRY_LATER: - LOG.debug("Reinitialization event queueing will be retried in next iteration"); - return false; - - default: - LOG.info("Reinitialization event successfully queued"); - return false; - } - }) - .orElseGet(() -> { - LOG.error( - "ReInitializationResult is null, something went wrong in queueing reinitialization " + - "event"); - return true; - }); - - if (triggerFullSnapshot) { - fullSnapshot = true; - } - } - currentSequenceNumber = getCurrentOMDBSequenceNumber(); - LOG.debug("Updated current sequence number: {}", currentSequenceNumber); - loopCount++; - } catch (InterruptedException intEx) { - LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); - // We are updating the table even if it didn't run i.e. got interrupted beforehand - // to indicate that a task was supposed to run, but it didn't. - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - Thread.currentThread().interrupt(); - // Since thread is interrupted, we do not fall back to snapshot sync. - // Return with sync failed status. - return false; - } catch (Exception e) { - markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); - LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", - e.getMessage()); - fullSnapshot = true; - } - if (fullSnapshot) { - break; - } - } - LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, - getCurrentOMDBSequenceNumber() - fromSequenceNumber); - } - if (fullSnapshot) { - try { - executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); - } catch (InterruptedException intEx) { - LOG.error("OM DB Snapshot update sync thread was interrupted."); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - Thread.currentThread().interrupt(); - // Mark sync status as failed. - return false; - } catch (Exception e) { - metrics.incrNumSnapshotRequestsFailed(); - fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); - fullSnapshotReconTaskUpdater.recordRunCompletion(); - LOG.error("Unable to update Recon's metadata with new OM DB. ", e); - // Update health status in ReconContext - reconContext.updateHealthStatus(new AtomicBoolean(false)); - reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); - } + if (reInitializeTasksCalled.compareAndSet(false, true)) { + LOG.info("Calling reprocess on Recon tasks."); + reconTaskController.reInitializeTasks(omMetadataManager, null); + } else { + LOG.info("reInitializeTasks already called once; skipping."); } +// if (currentSequenceNumber <= 0) { +// fullSnapshot = true; +// } else { +// // Get updates from OM and apply to local Recon OM DB and update task status in table +// deltaReconTaskStatusUpdater.recordRunStart(); +// int loopCount = 0; +// long fromSequenceNumber = currentSequenceNumber; +// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; +// /** +// * This loop will continue to fetch and apply OM DB updates and with every +// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. +// * It continues to fetch from OM till the lag, between OM DB WAL sequence number +// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. +// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. +// */ +// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { +// try (OMDBUpdatesHandler omdbUpdatesHandler = +// new OMDBUpdatesHandler(omMetadataManager)) { +// +// // If interrupt was previously signalled, +// // we should check for it before starting delta update sync. +// if (Thread.currentThread().isInterrupted()) { +// throw new InterruptedException("Thread interrupted during delta update."); +// } +// diffBetweenOMDbAndReconDBSeqNumber = +// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); +// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); +// // Keeping last updated sequence number for both full and delta tasks to be same +// // because sequence number of DB denotes and points to same OM DB copy of Recon, +// // even though two different tasks are updating the DB at different conditions, but +// // it tells the sync state with actual OM DB for the same Recon OM DB copy. +// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); +// deltaReconTaskStatusUpdater.recordRunCompletion(); +// fullSnapshotReconTaskUpdater.updateDetails(); +// // Update the current OM metadata manager in task controller +// reconTaskController.updateOMMetadataManager(omMetadataManager); +// +// // Pass on DB update events to tasks that are listening. +// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( +// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); +// +// // Check if task reinitialization is needed due to buffer overflow or task failures +// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); +// boolean tasksFailed = reconTaskController.hasTasksFailed(); +// +// if (bufferOverflowed || tasksFailed) { +// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? +// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : +// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; +// +// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", +// reason); +// +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// +// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally +// ReconTaskController.ReInitializationResult result = +// reconTaskController.queueReInitializationEvent(reason); +// +// //TODO: Create a metric to track this event buffer overflow or task failure event +// boolean triggerFullSnapshot = +// Optional.ofNullable(result) +// .map(r -> { +// switch (r) { +// case MAX_RETRIES_EXCEEDED: +// LOG.warn( +// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + +// "fallback"); +// return true; +// +// case RETRY_LATER: +// LOG.debug("Reinitialization event queueing will be retried in next iteration"); +// return false; +// +// default: +// LOG.info("Reinitialization event successfully queued"); +// return false; +// } +// }) +// .orElseGet(() -> { +// LOG.error( +// "ReInitializationResult is null, something went wrong in queueing reinitialization " + +// "event"); +// return true; +// }); +// +// if (triggerFullSnapshot) { +// fullSnapshot = true; +// } +// } +// currentSequenceNumber = getCurrentOMDBSequenceNumber(); +// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); +// loopCount++; +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); +// // We are updating the table even if it didn't run i.e. got interrupted beforehand +// // to indicate that a task was supposed to run, but it didn't. +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// Thread.currentThread().interrupt(); +// // Since thread is interrupted, we do not fall back to snapshot sync. +// // Return with sync failed status. +// return false; +// } catch (Exception e) { +// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); +// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", +// e.getMessage()); +// fullSnapshot = true; +// } +// if (fullSnapshot) { +// break; +// } +// } +// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, +// getCurrentOMDBSequenceNumber() - fromSequenceNumber); +// } + +// if (fullSnapshot) { +// try { +// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); +// } catch (InterruptedException intEx) { +// LOG.error("OM DB Snapshot update sync thread was interrupted."); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// Thread.currentThread().interrupt(); +// // Mark sync status as failed. +// return false; +// } catch (Exception e) { +// metrics.incrNumSnapshotRequestsFailed(); +// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); +// fullSnapshotReconTaskUpdater.recordRunCompletion(); +// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); +// // Update health status in ReconContext +// reconContext.updateHealthStatus(new AtomicBoolean(false)); +// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); +// } +// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java index 266beee7af3f..626376ac09a9 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java @@ -207,7 +207,7 @@ public static boolean reprocess(OMMetadataManager omMetadataManager, // Clean up shared resources SHARED_CONTAINER_KEY_COUNT_MAP.clear(); ReconConstants.CONTAINER_KEY_MAPPER_INITIALIZED.set(false); - LOG.debug("{}: Last task completed. Cleared shared map and reset initialization flag.", taskName); + LOG.info("{}: Last task completed. Cleared shared map and reset initialization flag.", taskName); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java index 627fee974b2a..c91a3d97775b 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/util/ParallelTableIteratorOperation.java @@ -171,6 +171,7 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, AtomicLong keyCounter = new AtomicLong(); AtomicLong prevLogCounter = new AtomicLong(); + Object logLock = new Object(); // ===== STEP 2: START ITERATOR THREADS ===== // For each segment boundary, create an iterator thread @@ -231,10 +232,10 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, } keyCounter.addAndGet(keyValues.size()); if (keyCounter.get() - prevLogCounter.get() > logCountThreshold) { - synchronized (keyCounter) { + synchronized (logLock) { if (keyCounter.get() - prevLogCounter.get() > logCountThreshold) { long cnt = keyCounter.get(); - LOG.info("Iterated through {} keys while performing task: {}", keyCounter.get(), taskName); + LOG.debug("Iterated through {} keys while performing task: {}", keyCounter.get(), taskName); prevLogCounter.set(cnt); } } From 1a26915255297cf3cbab43303dc39b954ff32ad5 Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 9 Dec 2025 12:12:34 +0530 Subject: [PATCH 41/49] Removed testing code --- .../impl/OzoneManagerServiceProviderImpl.java | 281 +++++++++--------- 1 file changed, 137 insertions(+), 144 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index c5c4a688327a..874049bfa89e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -118,7 +118,7 @@ public class OzoneManagerServiceProviderImpl private static final Logger LOG = LoggerFactory.getLogger(OzoneManagerServiceProviderImpl.class); private URLConnectionFactory connectionFactory; - private final AtomicBoolean reInitializeTasksCalled = new AtomicBoolean(false); + private File omSnapshotDBParentDir = null; private File reconDbDir = null; private String omDBSnapshotUrl; @@ -702,152 +702,145 @@ public boolean syncDataFromOM() { try { long currentSequenceNumber = getCurrentOMDBSequenceNumber(); LOG.info("Seq number of Recon's OM DB : {}", currentSequenceNumber); - boolean fullSnapshot = true; - + boolean fullSnapshot = false; - if (reInitializeTasksCalled.compareAndSet(false, true)) { - LOG.info("Calling reprocess on Recon tasks."); - reconTaskController.reInitializeTasks(omMetadataManager, null); + if (currentSequenceNumber <= 0) { + fullSnapshot = true; } else { - LOG.info("reInitializeTasks already called once; skipping."); + // Get updates from OM and apply to local Recon OM DB and update task status in table + deltaReconTaskStatusUpdater.recordRunStart(); + int loopCount = 0; + long fromSequenceNumber = currentSequenceNumber; + long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; + /** + * This loop will continue to fetch and apply OM DB updates and with every + * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. + * It continues to fetch from OM till the lag, between OM DB WAL sequence number + * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. + * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. + */ + while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { + try (OMDBUpdatesHandler omdbUpdatesHandler = + new OMDBUpdatesHandler(omMetadataManager)) { + + // If interrupt was previously signalled, + // we should check for it before starting delta update sync. + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Thread interrupted during delta update."); + } + diffBetweenOMDbAndReconDBSeqNumber = + getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); + deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); + // Keeping last updated sequence number for both full and delta tasks to be same + // because sequence number of DB denotes and points to same OM DB copy of Recon, + // even though two different tasks are updating the DB at different conditions, but + // it tells the sync state with actual OM DB for the same Recon OM DB copy. + deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); + deltaReconTaskStatusUpdater.recordRunCompletion(); + fullSnapshotReconTaskUpdater.updateDetails(); + // Update the current OM metadata manager in task controller + reconTaskController.updateOMMetadataManager(omMetadataManager); + + // Pass on DB update events to tasks that are listening. + reconTaskController.consumeOMEvents(new OMUpdateEventBatch( + omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); + + // Check if task reinitialization is needed due to buffer overflow or task failures + boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); + boolean tasksFailed = reconTaskController.hasTasksFailed(); + + if (bufferOverflowed || tasksFailed) { + ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? + ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : + ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; + + LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", + reason); + + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + + // Queue async reinitialization event - checkpoint creation and retry logic is handled internally + ReconTaskController.ReInitializationResult result = + reconTaskController.queueReInitializationEvent(reason); + + //TODO: Create a metric to track this event buffer overflow or task failure event + boolean triggerFullSnapshot = + Optional.ofNullable(result) + .map(r -> { + switch (r) { + case MAX_RETRIES_EXCEEDED: + LOG.warn( + "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + + "fallback"); + return true; + + case RETRY_LATER: + LOG.debug("Reinitialization event queueing will be retried in next iteration"); + return false; + + default: + LOG.info("Reinitialization event successfully queued"); + return false; + } + }) + .orElseGet(() -> { + LOG.error( + "ReInitializationResult is null, something went wrong in queueing reinitialization " + + "event"); + return true; + }); + + if (triggerFullSnapshot) { + fullSnapshot = true; + } + } + currentSequenceNumber = getCurrentOMDBSequenceNumber(); + LOG.debug("Updated current sequence number: {}", currentSequenceNumber); + loopCount++; + } catch (InterruptedException intEx) { + LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); + // We are updating the table even if it didn't run i.e. got interrupted beforehand + // to indicate that a task was supposed to run, but it didn't. + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + Thread.currentThread().interrupt(); + // Since thread is interrupted, we do not fall back to snapshot sync. + // Return with sync failed status. + return false; + } catch (Exception e) { + markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); + LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", + e.getMessage()); + fullSnapshot = true; + } + if (fullSnapshot) { + break; + } + } + LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, + getCurrentOMDBSequenceNumber() - fromSequenceNumber); + } + + if (fullSnapshot) { + try { + executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); + } catch (InterruptedException intEx) { + LOG.error("OM DB Snapshot update sync thread was interrupted."); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + Thread.currentThread().interrupt(); + // Mark sync status as failed. + return false; + } catch (Exception e) { + metrics.incrNumSnapshotRequestsFailed(); + fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); + fullSnapshotReconTaskUpdater.recordRunCompletion(); + LOG.error("Unable to update Recon's metadata with new OM DB. ", e); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); + } } -// if (currentSequenceNumber <= 0) { -// fullSnapshot = true; -// } else { -// // Get updates from OM and apply to local Recon OM DB and update task status in table -// deltaReconTaskStatusUpdater.recordRunStart(); -// int loopCount = 0; -// long fromSequenceNumber = currentSequenceNumber; -// long diffBetweenOMDbAndReconDBSeqNumber = deltaUpdateLimit + 1; -// /** -// * This loop will continue to fetch and apply OM DB updates and with every -// * OM DB fetch request, it will fetch {@code deltaUpdateLimit} count of DB updates. -// * It continues to fetch from OM till the lag, between OM DB WAL sequence number -// * and Recon OM DB snapshot WAL sequence number, is less than this lag threshold value. -// * In high OM write TPS cluster, this simulates continuous pull from OM without any delay. -// */ -// while (diffBetweenOMDbAndReconDBSeqNumber > omDBLagThreshold) { -// try (OMDBUpdatesHandler omdbUpdatesHandler = -// new OMDBUpdatesHandler(omMetadataManager)) { -// -// // If interrupt was previously signalled, -// // we should check for it before starting delta update sync. -// if (Thread.currentThread().isInterrupted()) { -// throw new InterruptedException("Thread interrupted during delta update."); -// } -// diffBetweenOMDbAndReconDBSeqNumber = -// getAndApplyDeltaUpdatesFromOM(currentSequenceNumber, omdbUpdatesHandler); -// deltaReconTaskStatusUpdater.setLastTaskRunStatus(0); -// // Keeping last updated sequence number for both full and delta tasks to be same -// // because sequence number of DB denotes and points to same OM DB copy of Recon, -// // even though two different tasks are updating the DB at different conditions, but -// // it tells the sync state with actual OM DB for the same Recon OM DB copy. -// deltaReconTaskStatusUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// fullSnapshotReconTaskUpdater.setLastUpdatedSeqNumber(getCurrentOMDBSequenceNumber()); -// deltaReconTaskStatusUpdater.recordRunCompletion(); -// fullSnapshotReconTaskUpdater.updateDetails(); -// // Update the current OM metadata manager in task controller -// reconTaskController.updateOMMetadataManager(omMetadataManager); -// -// // Pass on DB update events to tasks that are listening. -// reconTaskController.consumeOMEvents(new OMUpdateEventBatch( -// omdbUpdatesHandler.getEvents(), omdbUpdatesHandler.getLatestSequenceNumber()), omMetadataManager); -// -// // Check if task reinitialization is needed due to buffer overflow or task failures -// boolean bufferOverflowed = reconTaskController.hasEventBufferOverflowed(); -// boolean tasksFailed = reconTaskController.hasTasksFailed(); -// -// if (bufferOverflowed || tasksFailed) { -// ReconTaskReInitializationEvent.ReInitializationReason reason = bufferOverflowed ? -// ReconTaskReInitializationEvent.ReInitializationReason.BUFFER_OVERFLOW : -// ReconTaskReInitializationEvent.ReInitializationReason.TASK_FAILURES; -// -// LOG.warn("Detected condition for task reinitialization: {}, queueing async reinitialization event", -// reason); -// -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// -// // Queue async reinitialization event - checkpoint creation and retry logic is handled internally -// ReconTaskController.ReInitializationResult result = -// reconTaskController.queueReInitializationEvent(reason); -// -// //TODO: Create a metric to track this event buffer overflow or task failure event -// boolean triggerFullSnapshot = -// Optional.ofNullable(result) -// .map(r -> { -// switch (r) { -// case MAX_RETRIES_EXCEEDED: -// LOG.warn( -// "Reinitialization queue failures exceeded maximum retries, triggering full snapshot " + -// "fallback"); -// return true; -// -// case RETRY_LATER: -// LOG.debug("Reinitialization event queueing will be retried in next iteration"); -// return false; -// -// default: -// LOG.info("Reinitialization event successfully queued"); -// return false; -// } -// }) -// .orElseGet(() -> { -// LOG.error( -// "ReInitializationResult is null, something went wrong in queueing reinitialization " + -// "event"); -// return true; -// }); -// -// if (triggerFullSnapshot) { -// fullSnapshot = true; -// } -// } -// currentSequenceNumber = getCurrentOMDBSequenceNumber(); -// LOG.debug("Updated current sequence number: {}", currentSequenceNumber); -// loopCount++; -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Delta update sync thread was interrupted and delta sync failed."); -// // We are updating the table even if it didn't run i.e. got interrupted beforehand -// // to indicate that a task was supposed to run, but it didn't. -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// Thread.currentThread().interrupt(); -// // Since thread is interrupted, we do not fall back to snapshot sync. -// // Return with sync failed status. -// return false; -// } catch (Exception e) { -// markDeltaTaskStatusAsFailed(deltaReconTaskStatusUpdater); -// LOG.warn("Unable to get and apply delta updates from OM: {}, falling back to full snapshot", -// e.getMessage()); -// fullSnapshot = true; -// } -// if (fullSnapshot) { -// break; -// } -// } -// LOG.info("Delta updates received from OM : {} loops, {} records", loopCount, -// getCurrentOMDBSequenceNumber() - fromSequenceNumber); -// } - -// if (fullSnapshot) { -// try { -// executeFullSnapshot(fullSnapshotReconTaskUpdater, deltaReconTaskStatusUpdater); -// } catch (InterruptedException intEx) { -// LOG.error("OM DB Snapshot update sync thread was interrupted."); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// Thread.currentThread().interrupt(); -// // Mark sync status as failed. -// return false; -// } catch (Exception e) { -// metrics.incrNumSnapshotRequestsFailed(); -// fullSnapshotReconTaskUpdater.setLastTaskRunStatus(-1); -// fullSnapshotReconTaskUpdater.recordRunCompletion(); -// LOG.error("Unable to update Recon's metadata with new OM DB. ", e); -// // Update health status in ReconContext -// reconContext.updateHealthStatus(new AtomicBoolean(false)); -// reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); -// } -// } printOMDBMetaInfo(); } finally { isSyncDataFromOMRunning.set(false); From 0c9cd92199e045a9fd988f8b39ed944ae6b9c38a Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 9 Dec 2025 15:49:22 +0530 Subject: [PATCH 42/49] Fixed failing tests --- .../tasks/MultipartInfoInsightHandler.java | 14 ++++++---- .../recon/tasks/TestOmTableInsightTask.java | 27 +++++++++++++++++++ 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/MultipartInfoInsightHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/MultipartInfoInsightHandler.java index 3976919ab4b1..828192ec1277 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/MultipartInfoInsightHandler.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/MultipartInfoInsightHandler.java @@ -17,10 +17,12 @@ package org.apache.hadoop.ozone.recon.tasks; +import java.io.IOException; import java.util.Map; import org.apache.commons.lang3.tuple.Triple; 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.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo; import org.apache.hadoop.ozone.recon.api.types.ReconBasicOmKeyInfo; @@ -148,17 +150,19 @@ public void handleUpdateEvent(OMDBUpdateEvent event, String tabl * uploads in the backend. */ @Override - public Triple getTableSizeAndCount( - TableIterator> iterator) { + public Triple getTableSizeAndCount(String tableName, + OMMetadataManager omMetadataManager) throws IOException { long count = 0; long unReplicatedSize = 0; long replicatedSize = 0; - if (iterator != null) { + Table table = + (Table) omMetadataManager.getTable(tableName); + try (TableIterator> iterator = table.iterator()) { while (iterator.hasNext()) { - Table.KeyValue kv = iterator.next(); + Table.KeyValue kv = iterator.next(); if (kv != null && kv.getValue() != null) { - OmMultipartKeyInfo multipartKeyInfo = (OmMultipartKeyInfo) kv.getValue(); + OmMultipartKeyInfo multipartKeyInfo = kv.getValue(); for (PartKeyInfo partKeyInfo : multipartKeyInfo.getPartKeyInfoMap()) { ReconBasicOmKeyInfo omKeyInfo = ReconBasicOmKeyInfo.getFromProtobuf(partKeyInfo.getPartKeyInfo()); unReplicatedSize += omKeyInfo.getDataSize(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java index faa158bfab3d..4f64d27297b2 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOmTableInsightTask.java @@ -38,6 +38,8 @@ import static org.apache.ozone.recon.schema.generated.tables.GlobalStatsTable.GLOBAL_STATS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -53,6 +55,7 @@ import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TypedTable; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -346,6 +349,25 @@ public void testProcessForDeletedDirectoryTable() throws IOException { @Test public void testReprocessForCount() throws Exception { OMMetadataManager omMetadataManager = mock(OmMetadataManagerImpl.class); + + // Mock DBStore for getStore() calls + DBStore mockStore = mock(DBStore.class); + when(omMetadataManager.getStore()).thenReturn(mockStore); + + // Mock getDeletedTable() for DeletedKeysInsightHandler + TypedTable deletedTable = mock(TypedTable.class); + TypedTable.TypedTableIterator deletedIter = mock(TypedTable.TypedTableIterator.class); + when(deletedTable.iterator()).thenReturn(deletedIter); + when(omMetadataManager.getDeletedTable()).thenReturn(deletedTable); + when(deletedIter.hasNext()).thenReturn(true, true, true, true, true, false); + + RepeatedOmKeyInfo deletedKeyInfo = mock(RepeatedOmKeyInfo.class); + when(deletedKeyInfo.getTotalSize()).thenReturn(ImmutablePair.of(100L, 100L)); + when(deletedKeyInfo.getOmKeyInfoList()).thenReturn(Arrays.asList(mock(OmKeyInfo.class))); + + Table.KeyValue deletedKv = mock(Table.KeyValue.class); + when(deletedKv.getValue()).thenReturn(deletedKeyInfo); + when(deletedIter.next()).thenReturn(deletedKv); // Mock 5 rows in each table and test the count for (String tableName : omTableInsightTask.getTaskTables()) { @@ -353,8 +375,13 @@ public void testReprocessForCount() throws Exception { TypedTable.TypedTableIterator mockIter = mock(TypedTable.TypedTableIterator.class); when(table.iterator()).thenReturn(mockIter); + when(table.keyIterator()).thenReturn(mockIter); + when(table.getEstimatedKeyCount()).thenReturn(5L); when(omMetadataManager.getTable(tableName)).thenReturn(table); when(mockIter.hasNext()).thenReturn(true, true, true, true, true, false); + + // Mock DBStore.getTable() to return the same table + when(mockStore.getTable(eq(tableName), any(), any(), any())).thenAnswer(invocation -> table); final Table.KeyValue mockKeyValue = mock(Table.KeyValue.class); From 60a197a19c57cfa970925b928ce2a948fa04d2a1 Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 9 Dec 2025 20:12:08 +0530 Subject: [PATCH 43/49] Fixed TestConfigurationFieldsBase --- .../src/main/resources/ozone-default.xml | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 658a0d67d5ab..aed31414abfd 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -4318,6 +4318,43 @@ recon rocks DB containerKeyTable + + + ozone.recon.filesizecount.flush.db.max.threshold + 200000 + OZONE, RECON, PERFORMANCE + + Maximum threshold number of entries to hold in memory for File Size Count task in hashmap before flushing to + recon derby DB + + + + + ozone.recon.task.reprocess.max.iterators + 5 + OZONE, RECON, PERFORMANCE + + Maximum number of iterator threads to use for parallel table iteration during reprocess + + + + + ozone.recon.task.reprocess.max.workers + 20 + OZONE, RECON, PERFORMANCE + + Maximum number of worker threads to use for parallel table processing during reprocess + + + + + ozone.recon.task.reprocess.max.keys.in.memory + 2000 + OZONE, RECON, PERFORMANCE + + Maximum number of keys to batch in memory before handing to worker threads during parallel reprocess + + ozone.recon.heatmap.provider From e670c933f1e6b2d379fa5bb68b9841fd0640da67 Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 9 Dec 2025 22:12:02 +0530 Subject: [PATCH 44/49] Fixed container endpoint --- .../recon/api/TestContainerEndpoint.java | 36 ++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index 4794ecf1f309..320da191a9d5 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -47,6 +47,10 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import javax.ws.rs.WebApplicationException; @@ -77,6 +81,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.recon.ReconConstants; import org.apache.hadoop.ozone.recon.ReconTestInjector; import org.apache.hadoop.ozone.recon.api.types.ContainerDiscrepancyInfo; import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix; @@ -96,6 +101,7 @@ import org.apache.hadoop.ozone.recon.scm.ReconPipelineManager; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager; +import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperHelper; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; @@ -103,6 +109,7 @@ import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskFSO; import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskOBS; import org.apache.hadoop.ozone.recon.tasks.NSSummaryTaskWithFSO; +import org.apache.hadoop.ozone.recon.tasks.ReconOmTask; import org.apache.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates; import org.apache.ozone.recon.schema.generated.tables.pojos.UnhealthyContainers; import org.junit.jupiter.api.BeforeEach; @@ -297,14 +304,27 @@ public void setUp() throws Exception { reprocessContainerKeyMapper(); } - private void reprocessContainerKeyMapper() { + private void reprocessContainerKeyMapper() throws Exception { ContainerKeyMapperTaskOBS containerKeyMapperTaskOBS = new ContainerKeyMapperTaskOBS(reconContainerMetadataManager, omConfiguration); - containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager); - ContainerKeyMapperTaskFSO containerKeyMapperTaskFSO = new ContainerKeyMapperTaskFSO(reconContainerMetadataManager, omConfiguration); - containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager); + + // Run both tasks in parallel (like production) + ExecutorService executor = Executors.newFixedThreadPool(2); + try { + Future obsFuture = executor.submit( + () -> containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager)); + Future fsoFuture = executor.submit( + () -> containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager)); + + // Wait for both to complete + obsFuture.get(); + fsoFuture.get(); + } finally { + executor.shutdown(); + executor.awaitTermination(10, TimeUnit.SECONDS); + } } private void setUpFSOData() throws IOException { @@ -435,7 +455,7 @@ private OmKeyLocationInfoGroup getLocationInfoGroup1() { } @Test - public void testGetKeysForContainer() throws IOException { + public void testGetKeysForContainer() throws Exception { Response response = containerEndpoint.getKeysForContainer(1L, -1, ""); KeysResponse data = (KeysResponse) response.getEntity(); @@ -513,7 +533,7 @@ public void testGetKeysForContainer() throws IOException { } @Test - public void testGetKeysForContainerWithPrevKey() throws IOException { + public void testGetKeysForContainerWithPrevKey() throws Exception { // test if prev-key param works as expected Response response = containerEndpoint.getKeysForContainer( 1L, -1, "/sampleVol/bucketOne/key_one"); @@ -1370,7 +1390,7 @@ public void testGetContainerInsightsNonSCMContainers() @Test public void testGetContainerInsightsNonSCMContainersWithPrevKey() - throws IOException, TimeoutException { + throws Exception { // Add 3 more containers to OM making total container in OM to 5 String[] keys = {"key_three", "key_four", "key_five"}; @@ -1821,7 +1841,7 @@ private void setUpDuplicateFSOFileKeys() throws IOException { * and then verifies that the ContainerEndpoint returns two distinct key records. */ @Test - public void testDuplicateFSOKeysForContainerEndpoint() throws IOException { + public void testDuplicateFSOKeysForContainerEndpoint() throws Exception { // Set up duplicate FSO file keys. setUpDuplicateFSOFileKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = From a4d4f359c1e73c91b9e7c69fe68b9a40fb2fe9df Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 9 Dec 2025 23:33:31 +0530 Subject: [PATCH 45/49] Fixed checkstyle issues --- .../recon/api/TestContainerEndpoint.java | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index 320da191a9d5..0a9bb05d273d 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -81,7 +81,6 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; -import org.apache.hadoop.ozone.recon.ReconConstants; import org.apache.hadoop.ozone.recon.ReconTestInjector; import org.apache.hadoop.ozone.recon.api.types.ContainerDiscrepancyInfo; import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix; @@ -101,7 +100,6 @@ import org.apache.hadoop.ozone.recon.scm.ReconPipelineManager; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager; -import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperHelper; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; @@ -309,21 +307,21 @@ private void reprocessContainerKeyMapper() throws Exception { new ContainerKeyMapperTaskOBS(reconContainerMetadataManager, omConfiguration); ContainerKeyMapperTaskFSO containerKeyMapperTaskFSO = new ContainerKeyMapperTaskFSO(reconContainerMetadataManager, omConfiguration); - + // Run both tasks in parallel (like production) ExecutorService executor = Executors.newFixedThreadPool(2); try { - Future obsFuture = executor.submit( - () -> containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager)); - Future fsoFuture = executor.submit( - () -> containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager)); - - // Wait for both to complete - obsFuture.get(); - fsoFuture.get(); + Future obsFuture = executor.submit( + () -> containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager)); + Future fsoFuture = executor.submit( + () -> containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager)); + + // Wait for both to complete + obsFuture.get(); + fsoFuture.get(); } finally { - executor.shutdown(); - executor.awaitTermination(10, TimeUnit.SECONDS); + executor.shutdown(); + executor.awaitTermination(10, TimeUnit.SECONDS); } } From d60d4fe06575870e31d6f9805f810419f2b7a38d Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 10 Dec 2025 01:29:43 +0530 Subject: [PATCH 46/49] Fixed failing tests --- .../recon/TestReconContainerEndpoint.java | 2 ++ .../recon/api/TestContainerEndpoint.java | 34 +++++-------------- 2 files changed, 10 insertions(+), 26 deletions(-) diff --git a/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconContainerEndpoint.java b/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconContainerEndpoint.java index b278b14bc06e..d1d804f3a3a7 100644 --- a/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconContainerEndpoint.java +++ b/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconContainerEndpoint.java @@ -65,6 +65,8 @@ public void init() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT, OMConfigKeys.OZONE_BUCKET_LAYOUT_FILE_SYSTEM_OPTIMIZED); + // Configure multiple task threads for concurrent task execution + conf.setInt("ozone.recon.task.thread.count", 6); recon = new ReconService(conf); cluster = MiniOzoneCluster.newBuilder(conf) .setNumDatanodes(3) diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index 0a9bb05d273d..4794ecf1f309 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -47,10 +47,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import javax.ws.rs.WebApplicationException; @@ -107,7 +103,6 @@ import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskFSO; import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskOBS; import org.apache.hadoop.ozone.recon.tasks.NSSummaryTaskWithFSO; -import org.apache.hadoop.ozone.recon.tasks.ReconOmTask; import org.apache.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates; import org.apache.ozone.recon.schema.generated.tables.pojos.UnhealthyContainers; import org.junit.jupiter.api.BeforeEach; @@ -302,27 +297,14 @@ public void setUp() throws Exception { reprocessContainerKeyMapper(); } - private void reprocessContainerKeyMapper() throws Exception { + private void reprocessContainerKeyMapper() { ContainerKeyMapperTaskOBS containerKeyMapperTaskOBS = new ContainerKeyMapperTaskOBS(reconContainerMetadataManager, omConfiguration); + containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager); + ContainerKeyMapperTaskFSO containerKeyMapperTaskFSO = new ContainerKeyMapperTaskFSO(reconContainerMetadataManager, omConfiguration); - - // Run both tasks in parallel (like production) - ExecutorService executor = Executors.newFixedThreadPool(2); - try { - Future obsFuture = executor.submit( - () -> containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager)); - Future fsoFuture = executor.submit( - () -> containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager)); - - // Wait for both to complete - obsFuture.get(); - fsoFuture.get(); - } finally { - executor.shutdown(); - executor.awaitTermination(10, TimeUnit.SECONDS); - } + containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager); } private void setUpFSOData() throws IOException { @@ -453,7 +435,7 @@ private OmKeyLocationInfoGroup getLocationInfoGroup1() { } @Test - public void testGetKeysForContainer() throws Exception { + public void testGetKeysForContainer() throws IOException { Response response = containerEndpoint.getKeysForContainer(1L, -1, ""); KeysResponse data = (KeysResponse) response.getEntity(); @@ -531,7 +513,7 @@ public void testGetKeysForContainer() throws Exception { } @Test - public void testGetKeysForContainerWithPrevKey() throws Exception { + public void testGetKeysForContainerWithPrevKey() throws IOException { // test if prev-key param works as expected Response response = containerEndpoint.getKeysForContainer( 1L, -1, "/sampleVol/bucketOne/key_one"); @@ -1388,7 +1370,7 @@ public void testGetContainerInsightsNonSCMContainers() @Test public void testGetContainerInsightsNonSCMContainersWithPrevKey() - throws Exception { + throws IOException, TimeoutException { // Add 3 more containers to OM making total container in OM to 5 String[] keys = {"key_three", "key_four", "key_five"}; @@ -1839,7 +1821,7 @@ private void setUpDuplicateFSOFileKeys() throws IOException { * and then verifies that the ContainerEndpoint returns two distinct key records. */ @Test - public void testDuplicateFSOKeysForContainerEndpoint() throws Exception { + public void testDuplicateFSOKeysForContainerEndpoint() throws IOException { // Set up duplicate FSO file keys. setUpDuplicateFSOFileKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = From 6addd6db177e69187725ba7b63cbdae92a2003aa Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 10 Dec 2025 10:08:20 +0530 Subject: [PATCH 47/49] Fixed TestContainerEndpoint --- .../recon/api/TestContainerEndpoint.java | 44 +++++++++++++++---- 1 file changed, 36 insertions(+), 8 deletions(-) diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index 4794ecf1f309..0428b99f25e5 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -47,6 +47,10 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import javax.ws.rs.WebApplicationException; @@ -77,6 +81,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.recon.ReconConstants; import org.apache.hadoop.ozone.recon.ReconTestInjector; import org.apache.hadoop.ozone.recon.api.types.ContainerDiscrepancyInfo; import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix; @@ -97,12 +102,14 @@ import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperHelper; import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; import org.apache.hadoop.ozone.recon.spi.impl.StorageContainerServiceProviderImpl; import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskFSO; import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskOBS; import org.apache.hadoop.ozone.recon.tasks.NSSummaryTaskWithFSO; +import org.apache.hadoop.ozone.recon.tasks.ReconOmTask; import org.apache.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates; import org.apache.ozone.recon.schema.generated.tables.pojos.UnhealthyContainers; import org.junit.jupiter.api.BeforeEach; @@ -216,7 +223,15 @@ public void setUp() throws Exception { if (!isSetupDone) { initializeInjector(); isSetupDone = true; + } else { + // Clear shared state before subsequent tests to prevent data leakage + ContainerKeyMapperHelper.clearSharedContainerCountMap(); + ReconConstants.resetTableTruncatedFlags(); + + // Reinitialize container tables to clear RocksDB data + reconContainerMetadataManager.reinitWithNewContainerDataFromOm(Collections.emptyMap()); } + omConfiguration = new OzoneConfiguration(); List omKeyLocationInfoList = new ArrayList<>(); @@ -297,14 +312,27 @@ public void setUp() throws Exception { reprocessContainerKeyMapper(); } - private void reprocessContainerKeyMapper() { + private void reprocessContainerKeyMapper() throws Exception { ContainerKeyMapperTaskOBS containerKeyMapperTaskOBS = new ContainerKeyMapperTaskOBS(reconContainerMetadataManager, omConfiguration); - containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager); - ContainerKeyMapperTaskFSO containerKeyMapperTaskFSO = new ContainerKeyMapperTaskFSO(reconContainerMetadataManager, omConfiguration); - containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager); + + // Run both tasks in parallel (like production) + ExecutorService executor = Executors.newFixedThreadPool(2); + try { + Future obsFuture = executor.submit( + () -> containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager)); + Future fsoFuture = executor.submit( + () -> containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager)); + + // Wait for both to complete + obsFuture.get(); + fsoFuture.get(); + } finally { + executor.shutdown(); + executor.awaitTermination(10, TimeUnit.SECONDS); + } } private void setUpFSOData() throws IOException { @@ -435,7 +463,7 @@ private OmKeyLocationInfoGroup getLocationInfoGroup1() { } @Test - public void testGetKeysForContainer() throws IOException { + public void testGetKeysForContainer() throws Exception { Response response = containerEndpoint.getKeysForContainer(1L, -1, ""); KeysResponse data = (KeysResponse) response.getEntity(); @@ -513,7 +541,7 @@ public void testGetKeysForContainer() throws IOException { } @Test - public void testGetKeysForContainerWithPrevKey() throws IOException { + public void testGetKeysForContainerWithPrevKey() throws Exception { // test if prev-key param works as expected Response response = containerEndpoint.getKeysForContainer( 1L, -1, "/sampleVol/bucketOne/key_one"); @@ -1370,7 +1398,7 @@ public void testGetContainerInsightsNonSCMContainers() @Test public void testGetContainerInsightsNonSCMContainersWithPrevKey() - throws IOException, TimeoutException { + throws Exception { // Add 3 more containers to OM making total container in OM to 5 String[] keys = {"key_three", "key_four", "key_five"}; @@ -1821,7 +1849,7 @@ private void setUpDuplicateFSOFileKeys() throws IOException { * and then verifies that the ContainerEndpoint returns two distinct key records. */ @Test - public void testDuplicateFSOKeysForContainerEndpoint() throws IOException { + public void testDuplicateFSOKeysForContainerEndpoint() throws Exception { // Set up duplicate FSO file keys. setUpDuplicateFSOFileKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = From 1a1e8380a73d6f5d61e286de94c4d5f0d8fdf1dd Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 10 Dec 2025 10:14:36 +0530 Subject: [PATCH 48/49] Fixed checkstyle --- .../recon/api/TestContainerEndpoint.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index 0428b99f25e5..7df56a57be65 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -102,10 +102,10 @@ import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; -import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperHelper; import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; import org.apache.hadoop.ozone.recon.spi.impl.StorageContainerServiceProviderImpl; +import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperHelper; import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskFSO; import org.apache.hadoop.ozone.recon.tasks.ContainerKeyMapperTaskOBS; import org.apache.hadoop.ozone.recon.tasks.NSSummaryTaskWithFSO; @@ -227,11 +227,11 @@ public void setUp() throws Exception { // Clear shared state before subsequent tests to prevent data leakage ContainerKeyMapperHelper.clearSharedContainerCountMap(); ReconConstants.resetTableTruncatedFlags(); - + // Reinitialize container tables to clear RocksDB data reconContainerMetadataManager.reinitWithNewContainerDataFromOm(Collections.emptyMap()); } - + omConfiguration = new OzoneConfiguration(); List omKeyLocationInfoList = new ArrayList<>(); @@ -317,21 +317,21 @@ private void reprocessContainerKeyMapper() throws Exception { new ContainerKeyMapperTaskOBS(reconContainerMetadataManager, omConfiguration); ContainerKeyMapperTaskFSO containerKeyMapperTaskFSO = new ContainerKeyMapperTaskFSO(reconContainerMetadataManager, omConfiguration); - + // Run both tasks in parallel (like production) ExecutorService executor = Executors.newFixedThreadPool(2); try { - Future obsFuture = executor.submit( - () -> containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager)); - Future fsoFuture = executor.submit( - () -> containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager)); - - // Wait for both to complete - obsFuture.get(); - fsoFuture.get(); + Future obsFuture = executor.submit( + () -> containerKeyMapperTaskOBS.reprocess(reconOMMetadataManager)); + Future fsoFuture = executor.submit( + () -> containerKeyMapperTaskFSO.reprocess(reconOMMetadataManager)); + + // Wait for both to complete + obsFuture.get(); + fsoFuture.get(); } finally { - executor.shutdown(); - executor.awaitTermination(10, TimeUnit.SECONDS); + executor.shutdown(); + executor.awaitTermination(10, TimeUnit.SECONDS); } } From ca379ff5575a2a887f5541a2fd0fc0228bf40793 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 10 Dec 2025 10:39:00 +0530 Subject: [PATCH 49/49] Fixed TestReconAndAdminContainerCLI --- .../hadoop/ozone/recon/TestReconAndAdminContainerCLI.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconAndAdminContainerCLI.java b/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconAndAdminContainerCLI.java index 70be91b78624..52998a45c267 100644 --- a/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconAndAdminContainerCLI.java +++ b/hadoop-ozone/integration-test-recon/src/test/java/org/apache/hadoop/ozone/recon/TestReconAndAdminContainerCLI.java @@ -457,6 +457,8 @@ private static void setupConfigKeys() { 1, SECONDS); CONF.setTimeDuration(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, 0, SECONDS); + // Configure multiple task threads for concurrent task execution + CONF.setInt("ozone.recon.task.thread.count", 6); CONF.set(OzoneConfigKeys.OZONE_SCM_CLOSE_CONTAINER_WAIT_DURATION, "2s"); CONF.set(ScmConfigKeys.OZONE_SCM_PIPELINE_SCRUB_INTERVAL, "2s"); CONF.set(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, "5s");