From 9b4b59554084e69f6db2efeefbea1bf071a2adbc Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 11 Dec 2025 00:42:35 +0530 Subject: [PATCH 01/20] HDDS-14121. Parallelize NSSummaryTask tree rebuild. --- .../ozone/recon/tasks/NSSummaryTask.java | 19 +- .../tasks/NSSummaryTaskDbEventHandler.java | 148 ++++++++---- .../recon/tasks/NSSummaryTaskWithFSO.java | 217 +++++++++++++++--- .../recon/tasks/NSSummaryTaskWithLegacy.java | 11 +- .../recon/tasks/NSSummaryTaskWithOBS.java | 11 +- .../ozone/recon/tasks/OmTableInsightTask.java | 2 +- 6 files changed, 327 insertions(+), 81 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 6b4a5cce0e71..9081cee9783a 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,6 +41,7 @@ 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.slf4j.Logger; @@ -101,18 +102,30 @@ public NSSummaryTask(ReconNamespaceSummaryManager this.reconNamespaceSummaryManager = reconNamespaceSummaryManager; this.reconOMMetadataManager = reconOMMetadataManager; this.ozoneConfiguration = ozoneConfiguration; + 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 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 6d5d42f0880f..238fcf1a542d 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 @@ -62,9 +62,48 @@ public ReconOMMetadataManager getReconOMMetadataManager() { private void updateNSSummariesToDB(Map nsSummaryMap, Collection objectIdsToBeDeleted) throws IOException { try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { + // Read-Modify-Write for each entry to prevent race conditions for (Map.Entry entry : nsSummaryMap.entrySet()) { try { - reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, entry.getKey(), entry.getValue()); + Long parentId = entry.getKey(); + NSSummary deltaSummary = entry.getValue(); + + // Step 1: READ existing value from DB + NSSummary existingSummary = reconNamespaceSummaryManager.getNSSummary(parentId); + + if (existingSummary == null) { + // First time - just write the delta as-is + existingSummary = deltaSummary; + } else { + // Step 2: MODIFY - merge delta into existing + existingSummary.setNumOfFiles(existingSummary.getNumOfFiles() + deltaSummary.getNumOfFiles()); + existingSummary.setSizeOfFiles(existingSummary.getSizeOfFiles() + deltaSummary.getSizeOfFiles()); + existingSummary.setReplicatedSizeOfFiles( + existingSummary.getReplicatedSizeOfFiles() + deltaSummary.getReplicatedSizeOfFiles()); + + // Merge file size buckets + int[] existingBucket = existingSummary.getFileSizeBucket(); + int[] deltaBucket = deltaSummary.getFileSizeBucket(); + for (int i = 0; i < existingBucket.length; i++) { + existingBucket[i] += deltaBucket[i]; + } + existingSummary.setFileSizeBucket(existingBucket); + + // Merge child directory sets + existingSummary.getChildDir().addAll(deltaSummary.getChildDir()); + + // Preserve metadata if not set in existing + if ((existingSummary.getDirName() == null || existingSummary.getDirName().isEmpty()) && + (deltaSummary.getDirName() != null && !deltaSummary.getDirName().isEmpty())) { + existingSummary.setDirName(deltaSummary.getDirName()); + } + if (existingSummary.getParentId() == 0 && deltaSummary.getParentId() != 0) { + existingSummary.setParentId(deltaSummary.getParentId()); + } + } + + // Step 3: WRITE merged result back to DB + reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, parentId, existingSummary); } catch (IOException e) { LOG.error("Unable to write Namespace Summary data in Recon DB.", e); throw e; @@ -85,16 +124,20 @@ private void updateNSSummariesToDB(Map nsSummaryMap, Collection protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { + handlePutKeyEvent(keyInfo, nsSummaryMap, true); + } + + protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap, boolean allowDbRead) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); - // Try to get the NSSummary from our local map that maps NSSummaries to IDs + // Try to get from map 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 + if (nsSummary == null && allowDbRead) { + // Only read from DB during delta updates (process method) 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 + // Create new instance if not found nsSummary = new NSSummary(); } int[] fileBucket = nsSummary.getFileSizeBucket(); @@ -117,12 +160,19 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { + handlePutDirEvent(directoryInfo, nsSummaryMap, true); + } + + protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, + Map nsSummaryMap, + boolean allowDbRead) + throws IOException { long parentObjectId = directoryInfo.getParentObjectID(); long objectId = directoryInfo.getObjectID(); // write the dir name to the current directory @@ -130,11 +180,11 @@ protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, // Get or create the directory's NSSummary NSSummary curNSSummary = nsSummaryMap.get(objectId); - if (curNSSummary == null) { - // If we don't have it in this batch we try to get it from the DB + if (curNSSummary == null && allowDbRead) { + // Read from DB during delta updates (process method) curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId); } - + // Check if this directory already has content (files/subdirs) that need propagation boolean directoryAlreadyExists = (curNSSummary != null); long existingSizeOfFiles = directoryAlreadyExists ? curNSSummary.getSizeOfFiles() : 0; @@ -150,12 +200,12 @@ protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, // Get or create the parent's NSSummary NSSummary parentNSSummary = nsSummaryMap.get(parentObjectId); - if (parentNSSummary == null) { + if (parentNSSummary == null && allowDbRead) { + // Read from DB during delta updates (process method) parentNSSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } if (parentNSSummary == null) { - // If we don't have it locally and in the DB we create a new instance - // as this is a new ID + // Create new instance if not found parentNSSummary = new NSSummary(); } @@ -176,7 +226,7 @@ protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, // Propagate to grandparents and beyond propagateSizeUpwards(parentObjectId, existingSizeOfFiles, - existingReplicatedSizeOfFiles, existingNumOfFiles, nsSummaryMap); + existingReplicatedSizeOfFiles, existingNumOfFiles, nsSummaryMap, allowDbRead); } else { nsSummaryMap.put(parentObjectId, parentNSSummary); } @@ -185,15 +235,22 @@ protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, protected void handleDeleteKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { + handleDeleteKeyEvent(keyInfo, nsSummaryMap, true); + } + + protected void handleDeleteKeyEvent(OmKeyInfo keyInfo, + Map nsSummaryMap, + boolean allowDbRead) + throws IOException { long parentObjectId = keyInfo.getParentObjectID(); - // Try to get the NSSummary from our local map that maps NSSummaries to IDs + // Try to get from map 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 + if (nsSummary == null && allowDbRead) { + // Read from DB during delta updates (process method) nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } - // Just in case the OmKeyInfo isn't correctly written. + // Just in case the OmKeyInfo isn't correctly written if (nsSummary == null) { LOG.error("The namespace table is not correctly populated."); return; @@ -216,24 +273,31 @@ protected void handleDeleteKeyEvent(OmKeyInfo keyInfo, // Propagate upwards to all parents in the parent chain propagateSizeUpwards(parentObjectId, -keyInfo.getDataSize(), - -keyInfo.getReplicatedSize(), -1, nsSummaryMap); + -keyInfo.getReplicatedSize(), -1, nsSummaryMap, allowDbRead); } protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, Map nsSummaryMap) throws IOException { + handleDeleteDirEvent(directoryInfo, nsSummaryMap, true); + } + + protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, + Map nsSummaryMap, + boolean allowDbRead) + throws IOException { long deletedDirObjectId = directoryInfo.getObjectID(); long parentObjectId = directoryInfo.getParentObjectID(); // Get the deleted directory's NSSummary to extract its totals NSSummary deletedDirSummary = nsSummaryMap.get(deletedDirObjectId); - if (deletedDirSummary == null) { + if (deletedDirSummary == null && allowDbRead) { deletedDirSummary = reconNamespaceSummaryManager.getNSSummary(deletedDirObjectId); } // Get the parent directory's NSSummary NSSummary parentNsSummary = nsSummaryMap.get(parentObjectId); - if (parentNsSummary == null) { + if (parentNsSummary == null && allowDbRead) { parentNsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } @@ -265,7 +329,7 @@ protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, nsSummaryMap.put(parentObjectId, parentNsSummary); // Propagate to grandparents and beyond - propagateSizeUpwards(parentObjectId, -deletedSize, -deletedReplSize, -deletedNumFiles, nsSummaryMap); + propagateSizeUpwards(parentObjectId, -deletedSize, -deletedReplSize, -deletedNumFiles, nsSummaryMap, allowDbRead); // Set the deleted directory's parentId to 0 (unlink it) deletedDirSummary.setParentId(0); @@ -313,15 +377,17 @@ protected boolean flushAndCommitUpdatedNSToDB(Map nsSummaryMap, * reflect the total changes in their sizeOfFiles and numOfFiles fields. */ protected void propagateSizeUpwards(long objectId, long sizeChange, long replicatedSizeChange, - int countChange, Map nsSummaryMap) + int countChange, Map nsSummaryMap, + boolean allowDbRead) throws IOException { // Get the current directory's NSSummary NSSummary nsSummary = nsSummaryMap.get(objectId); - if (nsSummary == null) { + if (nsSummary == null && allowDbRead) { + // Read from DB during delta updates nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId); } if (nsSummary == null) { - return; // No more parents to update + return; // Not found, stop propagation } // Continue propagating to parent @@ -329,23 +395,27 @@ protected void propagateSizeUpwards(long objectId, long sizeChange, long replica if (parentId != 0) { // Get parent's NSSummary NSSummary parentSummary = nsSummaryMap.get(parentId); - if (parentSummary == null) { + if (parentSummary == null && allowDbRead) { + // Read from DB during delta updates parentSummary = reconNamespaceSummaryManager.getNSSummary(parentId); } - if (parentSummary != null) { - // Update parent's totals - parentSummary.setSizeOfFiles(parentSummary.getSizeOfFiles() + sizeChange); - long parentReplSize = parentSummary.getReplicatedSizeOfFiles(); - if (parentReplSize < 0) { - parentReplSize = 0; - } - parentSummary.setReplicatedSizeOfFiles(parentReplSize + replicatedSizeChange); - parentSummary.setNumOfFiles(parentSummary.getNumOfFiles() + countChange); - nsSummaryMap.put(parentId, parentSummary); - - // Recursively propagate to grandparents - propagateSizeUpwards(parentId, sizeChange, replicatedSizeChange, countChange, nsSummaryMap); + if (parentSummary == null) { + // Create new parent entry if not found + parentSummary = new NSSummary(); } + + // Update parent's totals + parentSummary.setSizeOfFiles(parentSummary.getSizeOfFiles() + sizeChange); + long parentReplSize = parentSummary.getReplicatedSizeOfFiles(); + if (parentReplSize < 0) { + parentReplSize = 0; + } + parentSummary.setReplicatedSizeOfFiles(parentReplSize + replicatedSizeChange); + parentSummary.setNumOfFiles(parentSummary.getNumOfFiles() + countChange); + nsSummaryMap.put(parentId, parentSummary); + + // Recursively propagate to grandparents + propagateSizeUpwards(parentId, sizeChange, replicatedSizeChange, countChange, nsSummaryMap, allowDbRead); } } 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..862b2db87b4d 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,10 +30,12 @@ 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.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; import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; @@ -40,6 +43,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,15 +56,24 @@ public class NSSummaryTaskWithFSO extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithFSO.class); private final long nsSummaryFlushToDBMaxThreshold; + private final int maxIterators; + private final int maxWorkers; + private final int maxKeysInMemory; public NSSummaryTaskWithFSO(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; } // We listen to updates from FSO-enabled FileTable, DirTable, DeletedTable and DeletedDirTable @@ -225,44 +238,20 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { Map nsSummaryMap = new HashMap<>(); 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; - } - } - } + // Step 1: Process dirTable in parallel (establishes directory hierarchy) + Map dirSummaryMap = processDirTableInParallel(omMetadataManager); + if (dirSummaryMap == null) { + return false; } + nsSummaryMap.putAll(dirSummaryMap); - // 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; - } - } - } + // Step 2: Process fileTable in parallel (large table) and merge into base map + if (!processFileTableInParallel(omMetadataManager, nsSummaryMap)) { + return false; } - } catch (IOException ioEx) { - LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", - ioEx); + } catch (Exception ex) { + LOG.error("Unable to reprocess Namespace Summary data in Recon DB.", ex); return false; } // flush and commit left out keys at end @@ -273,4 +262,160 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { LOG.info("Completed a reprocess run of NSSummaryTaskWithFSO"); return true; } + + /** + * Process dirTable in parallel using per-worker maps. + * Returns the aggregated map of NSSummary objects. + */ + private Map processDirTableInParallel(OMMetadataManager omMetadataManager) { + Table dirTable = omMetadataManager.getDirectoryTable(); + + // Per-worker maps for lockless updates + Map> allWorkerMaps = new ConcurrentHashMap<>(); + + // Divide threshold by worker count so each worker flushes independently + final long perWorkerThreshold = Math.max(1, nsSummaryFlushToDBMaxThreshold / maxWorkers); + + // Lock for coordinating DB flush operations only + Object flushLock = new Object(); + + Function, Void> kvOperation = kv -> { + // Get this worker's private map + Map workerMap = allWorkerMaps.computeIfAbsent( + Thread.currentThread().getId(), k -> new HashMap<>()); + + try { + // Call with allowDbRead=false for reprocess (no DB reads) + handlePutDirEvent(kv.getValue(), workerMap, false); + + // Flush this worker's map when it reaches threshold + if (workerMap.size() >= perWorkerThreshold) { + synchronized (flushLock) { + if (!flushAndCommitNSToDB(workerMap)) { + throw new IOException("Failed to flush NSSummary map to DB"); + } + } + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return null; + }; + + try (ParallelTableIteratorOperation parallelIter = + new ParallelTableIteratorOperation<>(omMetadataManager, dirTable, StringCodec.get(), + maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + parallelIter.performTaskOnTableVals("NSSummaryTaskWithFSO-dirTable", null, null, kvOperation); + } catch (Exception ex) { + LOG.error("Unable to process dirTable in parallel", ex); + return null; + } + + // Merge all worker maps into a single map to return + return mergeWorkerMaps(allWorkerMaps.values()); + } + + /** + * Process fileTable in parallel using per-worker maps and merge into base map. + */ + private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, + Map baseMap) { + Table fileTable = omMetadataManager.getFileTable(); + + // Per-worker maps for lockless updates + Map> allWorkerMaps = new ConcurrentHashMap<>(); + + // Divide threshold by worker count so each worker flushes independently + final long perWorkerThreshold = Math.max(1, nsSummaryFlushToDBMaxThreshold / maxWorkers); + + // Lock for coordinating DB flush operations only + Object flushLock = new Object(); + + Function, Void> kvOperation = kv -> { + // Get this worker's private map + Map workerMap = allWorkerMaps.computeIfAbsent( + Thread.currentThread().getId(), k -> new HashMap<>()); + + try { + // Call with allowDbRead=false for reprocess (no DB reads) + handlePutKeyEvent(kv.getValue(), workerMap, false); + + // Flush this worker's map when it reaches threshold + if (workerMap.size() >= perWorkerThreshold) { + synchronized (flushLock) { + if (!flushAndCommitNSToDB(workerMap)) { + throw new IOException("Failed to flush NSSummary map to DB"); + } + } + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return null; + }; + + try (ParallelTableIteratorOperation parallelIter = + new ParallelTableIteratorOperation<>(omMetadataManager, fileTable, StringCodec.get(), + maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + parallelIter.performTaskOnTableVals("NSSummaryTaskWithFSO-fileTable", null, null, kvOperation); + } catch (Exception ex) { + LOG.error("Unable to process fileTable in parallel", ex); + return false; + } + + // Merge all worker maps into base map + mergeWorkerMapsIntoBase(allWorkerMaps.values(), baseMap); + return true; + } + + /** + * Merge collection of worker maps into a single new map. + */ + private Map mergeWorkerMaps(Collection> workerMaps) { + Map mergedMap = new HashMap<>(); + mergeWorkerMapsIntoBase(workerMaps, mergedMap); + return mergedMap; + } + + /** + * Merge worker maps into base map, combining NSSummary values. + */ + private void mergeWorkerMapsIntoBase(Collection> workerMaps, + Map baseMap) { + for (Map workerMap : workerMaps) { + for (Map.Entry entry : workerMap.entrySet()) { + Long parentId = entry.getKey(); + NSSummary workerSummary = entry.getValue(); + + // Get or create in base map + NSSummary baseSummary = baseMap.computeIfAbsent(parentId, k -> new NSSummary()); + + // Merge worker's data into base + baseSummary.setNumOfFiles(baseSummary.getNumOfFiles() + workerSummary.getNumOfFiles()); + baseSummary.setSizeOfFiles(baseSummary.getSizeOfFiles() + workerSummary.getSizeOfFiles()); + baseSummary.setReplicatedSizeOfFiles( + baseSummary.getReplicatedSizeOfFiles() + workerSummary.getReplicatedSizeOfFiles()); + + // Merge file size buckets + int[] baseBucket = baseSummary.getFileSizeBucket(); + int[] workerBucket = workerSummary.getFileSizeBucket(); + for (int i = 0; i < baseBucket.length; i++) { + baseBucket[i] += workerBucket[i]; + } + baseSummary.setFileSizeBucket(baseBucket); + + // Merge child directory sets + baseSummary.getChildDir().addAll(workerSummary.getChildDir()); + + // Note: dirName and parentId should be consistent across workers for same ID + if ((baseSummary.getDirName() == null || baseSummary.getDirName().isEmpty()) && + (workerSummary.getDirName() != null && !workerSummary.getDirName().isEmpty())) { + baseSummary.setDirName(workerSummary.getDirName()); + } + if (baseSummary.getParentId() == 0 && workerSummary.getParentId() != 0) { + baseSummary.setParentId(workerSummary.getParentId()); + } + } + } + } } 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..76f5ff220339 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 @@ -55,6 +55,9 @@ public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler { private final boolean enableFileSystemPaths; private final long nsSummaryFlushToDBMaxThreshold; + private final int maxIterators; + private final int maxWorkers; + private final int maxKeysInMemory; public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager reconNamespaceSummaryManager, @@ -62,7 +65,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 +76,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, 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..792dcb083845 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 @@ -49,14 +49,23 @@ public class NSSummaryTaskWithOBS extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithOBS.class); private final long nsSummaryFlushToDBMaxThreshold; + private final int maxIterators; + private final int maxWorkers; + private final int maxKeysInMemory; 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) { 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 8027966231de..5e694fd855fe 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 @@ -211,7 +211,7 @@ private void processTableInParallel(String tableName, OMMetadataManager omMetada parallelIter.performTaskOnTableVals(getTaskName(), null, null, kv -> { if (kv != null) { - count.incrementAndGet(); + count.incrementAndGet(); } return null; }); From 8c791bbcb95f869933e3ca5441c80abf79cf919a Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 11 Dec 2025 12:16:09 +0530 Subject: [PATCH 02/20] Made some changes to the test code --- .../hadoop/ozone/recon/tasks/NSSummaryTask.java | 6 +++--- .../ozone/recon/tasks/NSSummaryTaskWithLegacy.java | 11 +---------- .../hadoop/ozone/recon/api/TestContainerEndpoint.java | 6 +++--- .../recon/api/TestNSSummaryDiskUsageOrdering.java | 2 +- .../ozone/recon/api/TestNSSummaryEndpointWithFSO.java | 2 +- .../api/TestNSSummaryEndpointWithOBSAndLegacy.java | 2 +- .../ozone/recon/api/TestOmDBInsightEndPoint.java | 4 ++-- .../ozone/recon/api/TestOpenKeysSearchEndpoint.java | 2 +- .../ozone/recon/tasks/TestNSSummaryTaskWithFSO.java | 2 +- .../ozone/recon/tasks/TestNSSummaryTaskWithOBS.java | 2 +- .../tasks/TestNSSummaryTreePrecomputeValues.java | 2 +- .../ozone/recon/tasks/TestOmTableInsightTask.java | 2 +- 12 files changed, 17 insertions(+), 26 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 9081cee9783a..889d2c7b82b3 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 @@ -120,9 +120,9 @@ public NSSummaryTask(ReconNamespaceSummaryManager this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); - this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( - reconNamespaceSummaryManager, reconOMMetadataManager, - ozoneConfiguration, nsSummaryFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); + this.nsSummaryTaskWithLegacy = + new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration, + nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, maxIterators, maxWorkers, maxKeysInMemory); 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 76f5ff220339..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 @@ -55,9 +55,6 @@ public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler { private final boolean enableFileSystemPaths; private final long nsSummaryFlushToDBMaxThreshold; - private final int maxIterators; - private final int maxWorkers; - private final int maxKeysInMemory; public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager reconNamespaceSummaryManager, @@ -65,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 @@ -76,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, 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 7df56a57be65..98eed88d018d 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 @@ -509,7 +509,7 @@ public void testGetKeysForContainer() throws Exception { setUpFSOData(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); // Reprocess the container key mapper to ensure the latest mapping is used reprocessContainerKeyMapper(); @@ -597,7 +597,7 @@ public void testGetKeysForContainerWithPrevKey() throws Exception { reprocessContainerKeyMapper(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); response = containerEndpoint.getKeysForContainer(20L, -1, "/0/1/2/file7"); @@ -1854,7 +1854,7 @@ public void testDuplicateFSOKeysForContainerEndpoint() throws Exception { setUpDuplicateFSOFileKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); 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..3047458423a6 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,5,20,2000); 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 5b15235431f5..6c510850f61c 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 @@ -432,7 +432,7 @@ public void setUp() throws Exception { setUpMultiBlockReplicatedKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } 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 ba190bee4c35..c8d4c533e5f9 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 @@ -369,7 +369,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, 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..0dc47b7c0897 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 @@ -331,10 +331,10 @@ public void setUp() throws Exception { reconOMMetadataManager, ozoneConfiguration, 10); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); NSSummaryTaskWithFSO nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); 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..f1cd1aa9fdb6 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,5,20,2000); 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..2aed805eb4bf 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,5,20,2000); } 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..96b76d73836b 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,5,20,2000); } /** 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..a5303eff05d2 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,5,20,2000); // 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 4f64d27297b2..8d48e75b1813 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 @@ -154,7 +154,7 @@ private void initializeInjector() throws IOException { omTableInsightTask = new OmTableInsightTask( reconGlobalStatsManager, reconOMMetadataManager); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( - reconNamespaceSummaryManager, reconOMMetadataManager, 10); + reconNamespaceSummaryManager, reconOMMetadataManager, 10,5,20,2000); dslContext = getDslContext(); omTableInsightTask.setTables(omTableInsightTask.getTaskTables()); From 22ce3e919aa0bce143f571151e1dffc3b4deb76c Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 11 Dec 2025 19:34:59 +0530 Subject: [PATCH 03/20] Reverted back to the old methods and created new methods for the reprocess --- .../ozone/recon/ReconControllerModule.java | 10 +- .../impl/OzoneManagerServiceProviderImpl.java | 281 +++++++++--------- .../tasks/NSSummaryTaskDbEventHandler.java | 201 +++++++++---- .../recon/tasks/NSSummaryTaskWithFSO.java | 8 +- .../recon/tasks/NSSummaryTaskWithLegacy.java | 6 +- .../recon/tasks/NSSummaryTaskWithOBS.java | 13 +- .../util/ParallelTableIteratorOperation.java | 2 +- 7 files changed, 296 insertions(+), 225 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..cd506852e0c5 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,11 +139,11 @@ 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(OmTableInsightTask.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); } } 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 d4edd2894f9d..e9b5afe6685b 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 @@ -113,7 +113,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; @@ -711,145 +711,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/NSSummaryTaskDbEventHandler.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java index 238fcf1a542d..aaea67744c76 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 @@ -124,16 +124,11 @@ private void updateNSSummariesToDB(Map nsSummaryMap, Collection protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { - handlePutKeyEvent(keyInfo, nsSummaryMap, true); - } - - protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap, boolean allowDbRead) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); // Try to get from map NSSummary nsSummary = nsSummaryMap.get(parentObjectId); - if (nsSummary == null && allowDbRead) { - // Only read from DB during delta updates (process method) + if (nsSummary == null) { + // Read from DB during delta updates (process method) nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } if (nsSummary == null) { @@ -160,18 +155,47 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) - throws IOException { - handlePutDirEvent(directoryInfo, nsSummaryMap, true); + /** + * Handle PUT key event during reprocess operation. + * Does not read from DB, only works with in-memory map. + */ + protected void handlePutKeyEventReprocess(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { + long parentObjectId = keyInfo.getParentObjectID(); + // Get from map only, no DB reads during reprocess + NSSummary nsSummary = nsSummaryMap.get(parentObjectId); + if (nsSummary == null) { + // Create new instance if not found + 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()); + + ++fileBucket[binIndex]; + nsSummary.setFileSizeBucket(fileBucket); + nsSummaryMap.put(parentObjectId, nsSummary); + + // Propagate upwards using reprocess-specific method + propagateSizeUpwardsReprocess(parentObjectId, keyInfo.getDataSize(), keyInfo.getReplicatedSize(), 1, nsSummaryMap); } protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, - Map nsSummaryMap, - boolean allowDbRead) + Map nsSummaryMap) throws IOException { long parentObjectId = directoryInfo.getParentObjectID(); long objectId = directoryInfo.getObjectID(); @@ -180,11 +204,11 @@ protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, // Get or create the directory's NSSummary NSSummary curNSSummary = nsSummaryMap.get(objectId); - if (curNSSummary == null && allowDbRead) { - // Read from DB during delta updates (process method) + if (curNSSummary == null) { + // If we don't have it in this batch we try to get it from the DB curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId); } - + // Check if this directory already has content (files/subdirs) that need propagation boolean directoryAlreadyExists = (curNSSummary != null); long existingSizeOfFiles = directoryAlreadyExists ? curNSSummary.getSizeOfFiles() : 0; @@ -200,12 +224,12 @@ protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, // Get or create the parent's NSSummary NSSummary parentNSSummary = nsSummaryMap.get(parentObjectId); - if (parentNSSummary == null && allowDbRead) { - // Read from DB during delta updates (process method) + if (parentNSSummary == null) { parentNSSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } if (parentNSSummary == null) { - // Create new instance if not found + // If we don't have it locally and in the DB we create a new instance + // as this is a new ID parentNSSummary = new NSSummary(); } @@ -226,31 +250,82 @@ protected void handlePutDirEvent(OmDirectoryInfo directoryInfo, // Propagate to grandparents and beyond propagateSizeUpwards(parentObjectId, existingSizeOfFiles, - existingReplicatedSizeOfFiles, existingNumOfFiles, nsSummaryMap, allowDbRead); + existingReplicatedSizeOfFiles, existingNumOfFiles, nsSummaryMap); } else { nsSummaryMap.put(parentObjectId, parentNSSummary); } } - protected void handleDeleteKeyEvent(OmKeyInfo keyInfo, - Map nsSummaryMap) + /** + * Handle PUT directory event during reprocess operation. + * Does not read from DB, only works with in-memory map. + */ + protected void handlePutDirEventReprocess(OmDirectoryInfo directoryInfo, + Map nsSummaryMap) throws IOException { - handleDeleteKeyEvent(keyInfo, nsSummaryMap, true); + long parentObjectId = directoryInfo.getParentObjectID(); + long objectId = directoryInfo.getObjectID(); + // write the dir name to the current directory + String dirName = directoryInfo.getName(); + + // Get or create the directory's NSSummary (no DB reads during reprocess) + NSSummary curNSSummary = nsSummaryMap.get(objectId); + + // Check if this directory already has content (files/subdirs) that need propagation + boolean directoryAlreadyExists = (curNSSummary != null); + long existingSizeOfFiles = directoryAlreadyExists ? curNSSummary.getSizeOfFiles() : 0; + int existingNumOfFiles = directoryAlreadyExists ? curNSSummary.getNumOfFiles() : 0; + long existingReplicatedSizeOfFiles = directoryAlreadyExists ? curNSSummary.getReplicatedSizeOfFiles() : 0; + + if (!directoryAlreadyExists) { + curNSSummary = new NSSummary(); + } + curNSSummary.setDirName(dirName); + curNSSummary.setParentId(parentObjectId); + nsSummaryMap.put(objectId, curNSSummary); + + // Get or create the parent's NSSummary (no DB reads during reprocess) + NSSummary parentNSSummary = nsSummaryMap.get(parentObjectId); + if (parentNSSummary == null) { + // Create new instance if not found + parentNSSummary = new NSSummary(); + } + + // Add child directory to parent + parentNSSummary.addChildDir(objectId); + + // If the directory already existed with content, update immediate parent's stats + if (directoryAlreadyExists && (existingSizeOfFiles > 0 || existingNumOfFiles > 0)) { + parentNSSummary.setNumOfFiles(parentNSSummary.getNumOfFiles() + existingNumOfFiles); + parentNSSummary.setSizeOfFiles(parentNSSummary.getSizeOfFiles() + existingSizeOfFiles); + + long parentReplSize = parentNSSummary.getReplicatedSizeOfFiles(); + if (parentReplSize < 0) { + parentReplSize = 0; + } + parentNSSummary.setReplicatedSizeOfFiles(parentReplSize + existingReplicatedSizeOfFiles); + nsSummaryMap.put(parentObjectId, parentNSSummary); + + // Propagate to grandparents and beyond using reprocess-specific method + propagateSizeUpwardsReprocess(parentObjectId, existingSizeOfFiles, + existingReplicatedSizeOfFiles, existingNumOfFiles, nsSummaryMap); + } else { + nsSummaryMap.put(parentObjectId, parentNSSummary); + } } protected void handleDeleteKeyEvent(OmKeyInfo keyInfo, - Map nsSummaryMap, - boolean allowDbRead) + Map nsSummaryMap) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); - // Try to get from map + // Try to get the NSSummary from our local map that maps NSSummaries to IDs NSSummary nsSummary = nsSummaryMap.get(parentObjectId); - if (nsSummary == null && allowDbRead) { - // Read from DB during delta updates (process method) + if (nsSummary == null) { + // If we don't have it in this batch we try to get it from the DB nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } - // Just in case the OmKeyInfo isn't correctly written + // Just in case the OmKeyInfo isn't correctly written. if (nsSummary == null) { LOG.error("The namespace table is not correctly populated."); return; @@ -273,31 +348,24 @@ protected void handleDeleteKeyEvent(OmKeyInfo keyInfo, // Propagate upwards to all parents in the parent chain propagateSizeUpwards(parentObjectId, -keyInfo.getDataSize(), - -keyInfo.getReplicatedSize(), -1, nsSummaryMap, allowDbRead); + -keyInfo.getReplicatedSize(), -1, nsSummaryMap); } protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, Map nsSummaryMap) throws IOException { - handleDeleteDirEvent(directoryInfo, nsSummaryMap, true); - } - - protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, - Map nsSummaryMap, - boolean allowDbRead) - throws IOException { long deletedDirObjectId = directoryInfo.getObjectID(); long parentObjectId = directoryInfo.getParentObjectID(); // Get the deleted directory's NSSummary to extract its totals NSSummary deletedDirSummary = nsSummaryMap.get(deletedDirObjectId); - if (deletedDirSummary == null && allowDbRead) { + if (deletedDirSummary == null) { deletedDirSummary = reconNamespaceSummaryManager.getNSSummary(deletedDirObjectId); } // Get the parent directory's NSSummary NSSummary parentNsSummary = nsSummaryMap.get(parentObjectId); - if (parentNsSummary == null && allowDbRead) { + if (parentNsSummary == null) { parentNsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } @@ -329,7 +397,7 @@ protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, nsSummaryMap.put(parentObjectId, parentNsSummary); // Propagate to grandparents and beyond - propagateSizeUpwards(parentObjectId, -deletedSize, -deletedReplSize, -deletedNumFiles, nsSummaryMap, allowDbRead); + propagateSizeUpwards(parentObjectId, -deletedSize, -deletedReplSize, -deletedNumFiles, nsSummaryMap); // Set the deleted directory's parentId to 0 (unlink it) deletedDirSummary.setParentId(0); @@ -377,17 +445,15 @@ protected boolean flushAndCommitUpdatedNSToDB(Map nsSummaryMap, * reflect the total changes in their sizeOfFiles and numOfFiles fields. */ protected void propagateSizeUpwards(long objectId, long sizeChange, long replicatedSizeChange, - int countChange, Map nsSummaryMap, - boolean allowDbRead) + int countChange, Map nsSummaryMap) throws IOException { // Get the current directory's NSSummary NSSummary nsSummary = nsSummaryMap.get(objectId); - if (nsSummary == null && allowDbRead) { - // Read from DB during delta updates + if (nsSummary == null) { nsSummary = reconNamespaceSummaryManager.getNSSummary(objectId); } if (nsSummary == null) { - return; // Not found, stop propagation + return; // No more parents to update } // Continue propagating to parent @@ -395,28 +461,35 @@ protected void propagateSizeUpwards(long objectId, long sizeChange, long replica if (parentId != 0) { // Get parent's NSSummary NSSummary parentSummary = nsSummaryMap.get(parentId); - if (parentSummary == null && allowDbRead) { - // Read from DB during delta updates - parentSummary = reconNamespaceSummaryManager.getNSSummary(parentId); - } if (parentSummary == null) { - // Create new parent entry if not found - parentSummary = new NSSummary(); + parentSummary = reconNamespaceSummaryManager.getNSSummary(parentId); } - - // Update parent's totals - parentSummary.setSizeOfFiles(parentSummary.getSizeOfFiles() + sizeChange); - long parentReplSize = parentSummary.getReplicatedSizeOfFiles(); - if (parentReplSize < 0) { - parentReplSize = 0; + if (parentSummary != null) { + // Update parent's totals + parentSummary.setSizeOfFiles(parentSummary.getSizeOfFiles() + sizeChange); + long parentReplSize = parentSummary.getReplicatedSizeOfFiles(); + if (parentReplSize < 0) { + parentReplSize = 0; + } + parentSummary.setReplicatedSizeOfFiles(parentReplSize + replicatedSizeChange); + parentSummary.setNumOfFiles(parentSummary.getNumOfFiles() + countChange); + nsSummaryMap.put(parentId, parentSummary); + + // Recursively propagate to grandparents + propagateSizeUpwards(parentId, sizeChange, replicatedSizeChange, countChange, nsSummaryMap); } - parentSummary.setReplicatedSizeOfFiles(parentReplSize + replicatedSizeChange); - parentSummary.setNumOfFiles(parentSummary.getNumOfFiles() + countChange); - nsSummaryMap.put(parentId, parentSummary); - - // Recursively propagate to grandparents - propagateSizeUpwards(parentId, sizeChange, replicatedSizeChange, countChange, nsSummaryMap, allowDbRead); } } + /** + * Propagates size and count changes upwards through the parent chain during reprocess. + * Used by reprocess method - does NOT read from DB, only works with in-memory map. + * This method will be implemented to handle the specific requirements of reprocess operation. + */ + protected void propagateSizeUpwardsReprocess(long objectId, long sizeChange, long replicatedSizeChange, + int countChange, Map nsSummaryMap) + throws IOException { + // TODO: Implementation will be added to handle reprocess-specific propagation logic + } + } 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 862b2db87b4d..cf141cefa5ab 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 @@ -285,8 +285,8 @@ private Map processDirTableInParallel(OMMetadataManager omMetad Thread.currentThread().getId(), k -> new HashMap<>()); try { - // Call with allowDbRead=false for reprocess (no DB reads) - handlePutDirEvent(kv.getValue(), workerMap, false); + // Use reprocess-specific method (no DB reads) + handlePutDirEventReprocess(kv.getValue(), workerMap); // Flush this worker's map when it reaches threshold if (workerMap.size() >= perWorkerThreshold) { @@ -337,8 +337,8 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, Thread.currentThread().getId(), k -> new HashMap<>()); try { - // Call with allowDbRead=false for reprocess (no DB reads) - handlePutKeyEvent(kv.getValue(), workerMap, false); + // Use reprocess-specific method (no DB reads) + handlePutKeyEventReprocess(kv.getValue(), workerMap); // Flush this worker's map when it reaches threshold if (workerMap.size() >= perWorkerThreshold) { 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..50a5e7557b8b 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 @@ -287,14 +287,14 @@ public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) { .setObjectID(keyInfo.getObjectID()) .setParentObjectID(keyInfo.getParentObjectID()) .build(); - handlePutDirEvent(directoryInfo, nsSummaryMap); + handlePutDirEventReprocess(directoryInfo, nsSummaryMap); } else { - handlePutKeyEvent(keyInfo, nsSummaryMap); + handlePutKeyEventReprocess(keyInfo, nsSummaryMap); } } else { // The LEGACY bucket is an object store bucket. setParentBucketId(keyInfo); - handlePutKeyEvent(keyInfo, nsSummaryMap); + handlePutKeyEventReprocess(keyInfo, nsSummaryMap); } if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { if (!flushAndCommitNSToDB(nsSummaryMap)) { 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 792dcb083845..de5ea28daac7 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 @@ -49,23 +49,14 @@ public class NSSummaryTaskWithOBS extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithOBS.class); private final long nsSummaryFlushToDBMaxThreshold; - private final int maxIterators; - private final int maxWorkers; - private final int maxKeysInMemory; 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) { @@ -99,7 +90,7 @@ public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { setKeyParentID(keyInfo); - handlePutKeyEvent(keyInfo, nsSummaryMap); + handlePutKeyEventReprocess(keyInfo, nsSummaryMap); if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { if (!flushAndCommitNSToDB(nsSummaryMap)) { return false; 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 c91a3d97775b..9519394482d0 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 @@ -235,7 +235,7 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, synchronized (logLock) { if (keyCounter.get() - prevLogCounter.get() > logCountThreshold) { long cnt = keyCounter.get(); - LOG.debug("Iterated through {} keys while performing task: {}", keyCounter.get(), taskName); + LOG.info("Iterated through {} keys while performing task: {}", keyCounter.get(), taskName); prevLogCounter.set(cnt); } } From 9fbe17e91dfc3b315848986d2dfd7b04c3b97a98 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 12 Dec 2025 21:15:58 +0530 Subject: [PATCH 04/20] Fixed reprocess --- .../recon/tasks/NSSummaryAsyncFlusher.java | 235 ++++++++++++++++++ .../recon/tasks/NSSummaryTaskWithFSO.java | 126 +++++----- 2 files changed, 305 insertions(+), 56 deletions(-) create mode 100644 hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java new file mode 100644 index 000000000000..35a47da8518a --- /dev/null +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -0,0 +1,235 @@ +/* + * 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; + +import java.io.Closeable; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; +import org.apache.hadoop.ozone.recon.api.types.NSSummary; +import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Async flusher for NSSummary maps with background thread. + * Workers submit their maps to a queue, background thread processes them. + */ +public class NSSummaryAsyncFlusher implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(NSSummaryAsyncFlusher.class); + + private final BlockingQueue> flushQueue; + private final Thread backgroundFlusher; + private final AtomicBoolean running = new AtomicBoolean(true); + private final ReconNamespaceSummaryManager reconNamespaceSummaryManager; + private final String taskName; + + // Poison pill to signal shutdown + private static final Map POISON_PILL = new HashMap<>(); + + public NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryManager, + String taskName, + int queueCapacity) { + this.reconNamespaceSummaryManager = reconNamespaceSummaryManager; + this.taskName = taskName; + this.flushQueue = new LinkedBlockingQueue<>(queueCapacity); + + this.backgroundFlusher = new Thread(this::flushLoop, taskName + "-AsyncFlusher"); + this.backgroundFlusher.setDaemon(true); + this.backgroundFlusher.start(); + LOG.info("{}: Started async flusher with queue capacity {}", taskName, queueCapacity); + } + + /** + * Submit a worker map for async flushing. + * Blocks if queue is full (natural backpressure). + */ + public void submitForFlush(Map workerMap) throws InterruptedException { + flushQueue.put(workerMap); + } + + /** + * Background thread loop that processes flush queue. + */ + private void flushLoop() { + while (running.get()) { + try { + Map workerMap = flushQueue.poll(1, TimeUnit.SECONDS); + + if (workerMap == null) { + continue; // Timeout, check running flag + } + + if (workerMap == POISON_PILL) { + LOG.info("{}: Received shutdown signal", taskName); + break; + } + + // Process this batch + flushWithPropagation(workerMap); + + } catch (InterruptedException e) { + LOG.info("{}: Flusher thread interrupted", taskName); + Thread.currentThread().interrupt(); + break; + } catch (Exception e) { + LOG.error("{}: Error in flush loop", taskName, e); + // Continue processing other batches + } + } + LOG.info("{}: Async flusher stopped", taskName); + } + + /** + * Flush worker map with propagation to ancestors. + */ + private void flushWithPropagation(Map workerMap) throws IOException { + Map mergedMap = new HashMap<>(); + + // For each immediate parent in worker map + for (Map.Entry entry : workerMap.entrySet()) { + long immediateParentId = entry.getKey(); + NSSummary delta = entry.getValue(); + + // Read immediate parent from DB or merged map + NSSummary parent = mergedMap.get(immediateParentId); + if (parent == null) { + parent = reconNamespaceSummaryManager.getNSSummary(immediateParentId); + if (parent == null) { + parent = new NSSummary(); + } + } + + // Apply delta to immediate parent + parent.setNumOfFiles(parent.getNumOfFiles() + delta.getNumOfFiles()); + parent.setSizeOfFiles(parent.getSizeOfFiles() + delta.getSizeOfFiles()); + parent.setReplicatedSizeOfFiles(parent.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); + + // Merge file size buckets + int[] parentBucket = parent.getFileSizeBucket(); + int[] deltaBucket = delta.getFileSizeBucket(); + for (int i = 0; i < parentBucket.length; i++) { + parentBucket[i] += deltaBucket[i]; + } + parent.setFileSizeBucket(parentBucket); + + // Merge child dirs + parent.getChildDir().addAll(delta.getChildDir()); + + // Store updated ACTUAL parent in merged map + mergedMap.put(immediateParentId, parent); + + // Propagate delta to ancestors (grandparent, great-grandparent, etc.) + propagateDeltaToAncestors(parent.getParentId(), delta, mergedMap); + } + + // Write merged map to DB (simple batch write, no more R-M-W needed) + writeToDb(mergedMap); + } + + /** + * Recursively propagate delta values up the ancestor chain. + * Pattern: Check merged map first (for updates in this batch), then DB, for ACTUAL value. + */ + private void propagateDeltaToAncestors(long ancestorId, NSSummary delta, + Map mergedMap) throws IOException { + // Base case: reached root + if (ancestorId == 0) { + return; + } + + // Get ACTUAL ancestor (check merged map first for most up-to-date, then DB) + NSSummary actualAncestor = mergedMap.get(ancestorId); + if (actualAncestor == null) { + actualAncestor = reconNamespaceSummaryManager.getNSSummary(ancestorId); + if (actualAncestor == null) { + // Ancestor not in DB yet, create new + actualAncestor = new NSSummary(); + } + } + + // Apply DELTA to ACTUAL ancestor + actualAncestor.setNumOfFiles(actualAncestor.getNumOfFiles() + delta.getNumOfFiles()); + actualAncestor.setSizeOfFiles(actualAncestor.getSizeOfFiles() + delta.getSizeOfFiles()); + actualAncestor.setReplicatedSizeOfFiles( + actualAncestor.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); + + // Store updated ACTUAL ancestor in merged map + mergedMap.put(ancestorId, actualAncestor); + + // Recursively propagate to next ancestor (grandparent, great-grandparent, etc.) + propagateDeltaToAncestors(actualAncestor.getParentId(), delta, mergedMap); + } + + /** + * Write merged map to DB using batch operation. + */ + private void writeToDb(Map mergedMap) throws IOException { + try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { + for (Map.Entry entry : mergedMap.entrySet()) { + reconNamespaceSummaryManager.batchStoreNSSummaries( + rdbBatchOperation, entry.getKey(), entry.getValue()); + } + reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation); + LOG.debug("{}: Flushed {} entries to DB", taskName, mergedMap.size()); + } catch (IOException e) { + LOG.error("{}: Failed to flush to DB", taskName, e); + throw e; + } + } + + @Override + public void close() throws IOException { + LOG.info("{}: Shutting down async flusher", taskName); + running.set(false); + + try { + // Send poison pill + flushQueue.offer(POISON_PILL, 5, TimeUnit.SECONDS); + + // Wait for background thread to finish + backgroundFlusher.join(30000); // 30 second timeout + + if (backgroundFlusher.isAlive()) { + LOG.warn("{}: Background flusher did not stop gracefully, interrupting", taskName); + backgroundFlusher.interrupt(); + } + + // Flush any remaining items + LOG.info("{}: Draining remaining {} items from queue", taskName, flushQueue.size()); + Map remaining; + while ((remaining = flushQueue.poll()) != null) { + if (remaining != POISON_PILL) { + flushWithPropagation(remaining); + } + } + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while shutting down async flusher", e); + } + + LOG.info("{}: Async flusher shut down complete", taskName); + } +} + 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 cf141cefa5ab..23cffd7611d9 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 @@ -235,69 +235,68 @@ private void handleUpdateOnFileTable(OMDBUpdateEvent nsSummaryMap = new HashMap<>(); - - try { - // Step 1: Process dirTable in parallel (establishes directory hierarchy) - Map dirSummaryMap = processDirTableInParallel(omMetadataManager); - if (dirSummaryMap == null) { + // We run reprocess in two phases with separate flushers so that directory + // skeletons are fully persisted before file updates rely on them. + final int queueCapacity = maxWorkers + 10; // small buffer beyond workers + + try (NSSummaryAsyncFlusher dirFlusher = + new NSSummaryAsyncFlusher(getReconNamespaceSummaryManager(), + "NSSummaryTaskWithFSO-dir", queueCapacity)) { + if (!processDirTableInParallel(omMetadataManager, dirFlusher)) { return false; } - nsSummaryMap.putAll(dirSummaryMap); + // dirFlusher.close() drains queue and persists skeletons before files stage + } catch (Exception ex) { + LOG.error("Unable to reprocess Namespace Summary data in Recon DB (dir phase).", ex); + return false; + } - // Step 2: Process fileTable in parallel (large table) and merge into base map - if (!processFileTableInParallel(omMetadataManager, nsSummaryMap)) { + try (NSSummaryAsyncFlusher fileFlusher = + new NSSummaryAsyncFlusher(getReconNamespaceSummaryManager(), + "NSSummaryTaskWithFSO-file", queueCapacity)) { + if (!processFileTableInParallel(omMetadataManager, fileFlusher)) { return false; } - + // fileFlusher.close() drains queue and persists file deltas with propagation } catch (Exception ex) { - LOG.error("Unable to reprocess Namespace Summary data in Recon DB.", ex); - return false; - } - // flush and commit left out keys at end - if (!flushAndCommitNSToDB(nsSummaryMap)) { - LOG.info("flushAndCommitNSToDB failed during reprocessWithFSO."); + LOG.error("Unable to reprocess Namespace Summary data in Recon DB (file phase).", ex); return false; } + LOG.info("Completed a reprocess run of NSSummaryTaskWithFSO"); return true; } /** - * Process dirTable in parallel using per-worker maps. - * Returns the aggregated map of NSSummary objects. + * Process dirTable in parallel using per-worker maps with async flushing. */ - private Map processDirTableInParallel(OMMetadataManager omMetadataManager) { + private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, + NSSummaryAsyncFlusher asyncFlusher) { Table dirTable = omMetadataManager.getDirectoryTable(); // Per-worker maps for lockless updates Map> allWorkerMaps = new ConcurrentHashMap<>(); - // Divide threshold by worker count so each worker flushes independently + // Divide threshold by worker count final long perWorkerThreshold = Math.max(1, nsSummaryFlushToDBMaxThreshold / maxWorkers); - - // Lock for coordinating DB flush operations only - Object flushLock = new Object(); Function, Void> kvOperation = kv -> { // Get this worker's private map - Map workerMap = allWorkerMaps.computeIfAbsent( - Thread.currentThread().getId(), k -> new HashMap<>()); + long threadId = Thread.currentThread().getId(); + Map workerMap = allWorkerMaps.computeIfAbsent(threadId, k -> new HashMap<>()); try { - // Use reprocess-specific method (no DB reads) + // Update immediate parent only, NO DB reads during reprocess handlePutDirEventReprocess(kv.getValue(), workerMap); - // Flush this worker's map when it reaches threshold + // Submit to async queue when threshold reached if (workerMap.size() >= perWorkerThreshold) { - synchronized (flushLock) { - if (!flushAndCommitNSToDB(workerMap)) { - throw new IOException("Failed to flush NSSummary map to DB"); - } - } + asyncFlusher.submitForFlush(workerMap); + // Get fresh map for this worker + allWorkerMaps.put(threadId, new HashMap<>()); } - } catch (IOException e) { - throw new UncheckedIOException(e); + } catch (Exception e) { + throw new RuntimeException(e); } return null; }; @@ -308,48 +307,54 @@ private Map processDirTableInParallel(OMMetadataManager omMetad parallelIter.performTaskOnTableVals("NSSummaryTaskWithFSO-dirTable", null, null, kvOperation); } catch (Exception ex) { LOG.error("Unable to process dirTable in parallel", ex); - return null; + return false; } - // Merge all worker maps into a single map to return - return mergeWorkerMaps(allWorkerMaps.values()); + // Submit any remaining worker maps + for (Map remainingMap : allWorkerMaps.values()) { + if (!remainingMap.isEmpty()) { + try { + asyncFlusher.submitForFlush(remainingMap); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } + } + + return true; } /** - * Process fileTable in parallel using per-worker maps and merge into base map. + * Process fileTable in parallel using per-worker maps with async flushing. */ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, - Map baseMap) { + NSSummaryAsyncFlusher asyncFlusher) { Table fileTable = omMetadataManager.getFileTable(); // Per-worker maps for lockless updates Map> allWorkerMaps = new ConcurrentHashMap<>(); - // Divide threshold by worker count so each worker flushes independently + // Divide threshold by worker count final long perWorkerThreshold = Math.max(1, nsSummaryFlushToDBMaxThreshold / maxWorkers); - // Lock for coordinating DB flush operations only - Object flushLock = new Object(); - Function, Void> kvOperation = kv -> { // Get this worker's private map - Map workerMap = allWorkerMaps.computeIfAbsent( - Thread.currentThread().getId(), k -> new HashMap<>()); + long threadId = Thread.currentThread().getId(); + Map workerMap = allWorkerMaps.computeIfAbsent(threadId, k -> new HashMap<>()); try { - // Use reprocess-specific method (no DB reads) + // Update immediate parent only, NO DB reads during reprocess handlePutKeyEventReprocess(kv.getValue(), workerMap); - // Flush this worker's map when it reaches threshold + // Submit to async queue when threshold reached if (workerMap.size() >= perWorkerThreshold) { - synchronized (flushLock) { - if (!flushAndCommitNSToDB(workerMap)) { - throw new IOException("Failed to flush NSSummary map to DB"); - } - } + asyncFlusher.submitForFlush(workerMap); + // Get fresh map for this worker + allWorkerMaps.put(threadId, new HashMap<>()); } - } catch (IOException e) { - throw new UncheckedIOException(e); + } catch (Exception e) { + throw new RuntimeException(e); } return null; }; @@ -363,8 +368,17 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, return false; } - // Merge all worker maps into base map - mergeWorkerMapsIntoBase(allWorkerMaps.values(), baseMap); + // Submit any remaining worker maps + for (Map remainingMap : allWorkerMaps.values()) { + if (!remainingMap.isEmpty()) { + try { + asyncFlusher.submitForFlush(remainingMap); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } + } return true; } From 8e505d4c5d8ff5d2532adf4c1d5d3c92497205a8 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 12 Dec 2025 21:16:11 +0530 Subject: [PATCH 05/20] Fixed reprocess --- .../src/main/resources/ozone-default.xml | 2 +- .../recon/tasks/NSSummaryAsyncFlusher.java | 93 ++++++++++++------- .../ozone/recon/tasks/NSSummaryTask.java | 3 +- .../tasks/NSSummaryTaskDbEventHandler.java | 1 + .../recon/tasks/NSSummaryTaskWithFSO.java | 16 ++-- ...TestNSSummaryEndpointWithOBSAndLegacy.java | 2 +- .../recon/api/TestOmDBInsightEndPoint.java | 2 +- .../recon/tasks/TestNSSummaryTaskWithOBS.java | 2 +- 8 files changed, 75 insertions(+), 46 deletions(-) diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index aed31414abfd..d4879cc97ba9 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -4318,7 +4318,7 @@ recon rocks DB containerKeyTable - + ozone.recon.filesizecount.flush.db.max.threshold 200000 diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 35a47da8518a..85db130a3dc3 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -44,8 +44,8 @@ public class NSSummaryAsyncFlusher implements Closeable { private final ReconNamespaceSummaryManager reconNamespaceSummaryManager; private final String taskName; - // Poison pill to signal shutdown - private static final Map POISON_PILL = new HashMap<>(); + // Sentinel value to signal shutdown + private static final Map SHUTDOWN_SIGNAL = new HashMap<>(); public NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryManager, String taskName, @@ -66,6 +66,7 @@ public NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryM */ public void submitForFlush(Map workerMap) throws InterruptedException { flushQueue.put(workerMap); + LOG.debug("{}: Submitted map with {} entries, queue size now {}", taskName, workerMap.size(), flushQueue.size()); } /** @@ -80,13 +81,15 @@ private void flushLoop() { continue; // Timeout, check running flag } - if (workerMap == POISON_PILL) { + if (workerMap == SHUTDOWN_SIGNAL) { LOG.info("{}: Received shutdown signal", taskName); break; } // Process this batch + LOG.debug("{}: Background thread processing batch with {} entries", taskName, workerMap.size()); flushWithPropagation(workerMap); + LOG.debug("{}: Background thread finished batch", taskName); } catch (InterruptedException e) { LOG.info("{}: Flusher thread interrupted", taskName); @@ -104,6 +107,7 @@ private void flushLoop() { * Flush worker map with propagation to ancestors. */ private void flushWithPropagation(Map workerMap) throws IOException { + LOG.debug("{}: Flush starting with {} entries", taskName, workerMap.size()); Map mergedMap = new HashMap<>(); // For each immediate parent in worker map @@ -111,40 +115,43 @@ private void flushWithPropagation(Map workerMap) throws IOExcep long immediateParentId = entry.getKey(); NSSummary delta = entry.getValue(); - // Read immediate parent from DB or merged map - NSSummary parent = mergedMap.get(immediateParentId); - if (parent == null) { - parent = reconNamespaceSummaryManager.getNSSummary(immediateParentId); - if (parent == null) { - parent = new NSSummary(); - } + // Get actual parent (check merged map first, then DB) + NSSummary actualParent = mergedMap.get(immediateParentId); + if (actualParent == null) { + actualParent = reconNamespaceSummaryManager.getNSSummary(immediateParentId); } - // Apply delta to immediate parent - parent.setNumOfFiles(parent.getNumOfFiles() + delta.getNumOfFiles()); - parent.setSizeOfFiles(parent.getSizeOfFiles() + delta.getSizeOfFiles()); - parent.setReplicatedSizeOfFiles(parent.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); - - // Merge file size buckets - int[] parentBucket = parent.getFileSizeBucket(); - int[] deltaBucket = delta.getFileSizeBucket(); - for (int i = 0; i < parentBucket.length; i++) { - parentBucket[i] += deltaBucket[i]; + if (actualParent == null) { + // Parent doesn't exist in DB yet - use delta as base (has metadata like dirName, parentId) + actualParent = delta; + } else { + // Parent exists in DB - merge delta into it + actualParent.setNumOfFiles(actualParent.getNumOfFiles() + delta.getNumOfFiles()); + actualParent.setSizeOfFiles(actualParent.getSizeOfFiles() + delta.getSizeOfFiles()); + actualParent.setReplicatedSizeOfFiles(actualParent.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); + + // Merge file size buckets + int[] actualBucket = actualParent.getFileSizeBucket(); + int[] deltaBucket = delta.getFileSizeBucket(); + for (int i = 0; i < actualBucket.length; i++) { + actualBucket[i] += deltaBucket[i]; + } + actualParent.setFileSizeBucket(actualBucket); + + // Merge child dirs + actualParent.getChildDir().addAll(delta.getChildDir()); } - parent.setFileSizeBucket(parentBucket); - - // Merge child dirs - parent.getChildDir().addAll(delta.getChildDir()); // Store updated ACTUAL parent in merged map - mergedMap.put(immediateParentId, parent); + mergedMap.put(immediateParentId, actualParent); // Propagate delta to ancestors (grandparent, great-grandparent, etc.) - propagateDeltaToAncestors(parent.getParentId(), delta, mergedMap); + propagateDeltaToAncestors(actualParent.getParentId(), delta, mergedMap); } // Write merged map to DB (simple batch write, no more R-M-W needed) writeToDb(mergedMap); + LOG.debug("{}: Flush completed, wrote {} entries", taskName, mergedMap.size()); } /** @@ -191,7 +198,7 @@ private void writeToDb(Map mergedMap) throws IOException { rdbBatchOperation, entry.getKey(), entry.getValue()); } reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation); - LOG.debug("{}: Flushed {} entries to DB", taskName, mergedMap.size()); + LOG.debug("{}: Wrote {} entries to DB", taskName, mergedMap.size()); } catch (IOException e) { LOG.error("{}: Failed to flush to DB", taskName, e); throw e; @@ -201,11 +208,23 @@ private void writeToDb(Map mergedMap) throws IOException { @Override public void close() throws IOException { LOG.info("{}: Shutting down async flusher", taskName); + + // Wait for queue to drain before shutting down + long waitStart = System.currentTimeMillis(); + while (!flushQueue.isEmpty() && (System.currentTimeMillis() - waitStart) < 10000) { + try { + Thread.sleep(100); // Give background thread time to process + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + running.set(false); try { - // Send poison pill - flushQueue.offer(POISON_PILL, 5, TimeUnit.SECONDS); + // Send shutdown signal + flushQueue.offer(SHUTDOWN_SIGNAL, 5, TimeUnit.SECONDS); // Wait for background thread to finish backgroundFlusher.join(30000); // 30 second timeout @@ -216,13 +235,23 @@ public void close() throws IOException { } // Flush any remaining items - LOG.info("{}: Draining remaining {} items from queue", taskName, flushQueue.size()); + int remainingCount = flushQueue.size(); + LOG.info("{}: Draining remaining {} items from queue", taskName, remainingCount); Map remaining; + int processed = 0; while ((remaining = flushQueue.poll()) != null) { - if (remaining != POISON_PILL) { - flushWithPropagation(remaining); + if (remaining != SHUTDOWN_SIGNAL) { + try { + LOG.info("{}: Draining item {} with {} entries", taskName, ++processed, remaining.size()); + flushWithPropagation(remaining); + LOG.info("{}: Successfully flushed item {}", taskName, processed); + } catch (Exception e) { + LOG.error("{}: Error flushing item {} during drain", taskName, processed, e); + // Continue draining other items + } } } + LOG.info("{}: Finished draining, processed {} of {} items", taskName, processed, remainingCount); } catch (InterruptedException e) { Thread.currentThread().interrupt(); 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 889d2c7b82b3..27f4f9da062f 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 @@ -124,8 +124,7 @@ public NSSummaryTask(ReconNamespaceSummaryManager new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, reconOMMetadataManager, 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 aaea67744c76..94ef14ee63c3 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 @@ -283,6 +283,7 @@ protected void handlePutDirEventReprocess(OmDirectoryInfo directoryInfo, curNSSummary.setDirName(dirName); curNSSummary.setParentId(parentObjectId); nsSummaryMap.put(objectId, curNSSummary); + LOG.info("Set dir metadata: objectId={}, dirName={}, parentId={}", objectId, dirName, parentObjectId); // Get or create the parent's NSSummary (no DB reads during reprocess) NSSummary parentNSSummary = nsSummaryMap.get(parentObjectId); 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 23cffd7611d9..5b337e9f3cef 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 @@ -260,12 +260,12 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { // fileFlusher.close() drains queue and persists file deltas with propagation } catch (Exception ex) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB (file phase).", ex); - return false; - } + return false; + } LOG.info("Completed a reprocess run of NSSummaryTaskWithFSO"); return true; - } + } /** * Process dirTable in parallel using per-worker maps with async flushing. @@ -273,7 +273,7 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, NSSummaryAsyncFlusher asyncFlusher) { Table dirTable = omMetadataManager.getDirectoryTable(); - + // Per-worker maps for lockless updates Map> allWorkerMaps = new ConcurrentHashMap<>(); @@ -317,10 +317,10 @@ private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, asyncFlusher.submitForFlush(remainingMap); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - return false; + return false; + } + } } - } - } return true; } @@ -375,7 +375,7 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, asyncFlusher.submitForFlush(remainingMap); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - return false; + return false; } } } 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 c8d4c533e5f9..ba190bee4c35 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 @@ -369,7 +369,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + reconOMMetadataManager, 10); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, 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 0dc47b7c0897..28a05fb71ca3 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 @@ -331,7 +331,7 @@ public void setUp() throws Exception { reconOMMetadataManager, ozoneConfiguration, 10); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + reconOMMetadataManager, 10); NSSummaryTaskWithFSO nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, 10,5,20,2000); 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 96b76d73836b..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,5,20,2000); + threshold); } /** From 18064511500de6b292e2791156a6a0d476ce584c Mon Sep 17 00:00:00 2001 From: arafat Date: Tue, 16 Dec 2025 14:37:42 +0530 Subject: [PATCH 06/20] Made the async flush thread code simple --- .../recon/tasks/NSSummaryAsyncFlusher.java | 97 +++++++------------ .../tasks/NSSummaryTaskDbEventHandler.java | 32 +----- .../recon/tasks/NSSummaryTaskWithFSO.java | 10 +- 3 files changed, 41 insertions(+), 98 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 85db130a3dc3..db639c5f05f3 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -24,7 +24,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; @@ -37,16 +37,20 @@ */ public class NSSummaryAsyncFlusher implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(NSSummaryAsyncFlusher.class); + + private enum FlushState { + RUNNING, + STOPPING, + STOPPED + } private final BlockingQueue> flushQueue; private final Thread backgroundFlusher; - private final AtomicBoolean running = new AtomicBoolean(true); + private final AtomicReference state = + new AtomicReference<>(FlushState.RUNNING); private final ReconNamespaceSummaryManager reconNamespaceSummaryManager; private final String taskName; - // Sentinel value to signal shutdown - private static final Map SHUTDOWN_SIGNAL = new HashMap<>(); - public NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryManager, String taskName, int queueCapacity) { @@ -73,25 +77,28 @@ public void submitForFlush(Map workerMap) throws InterruptedExc * Background thread loop that processes flush queue. */ private void flushLoop() { - while (running.get()) { + while (state.get() == FlushState.RUNNING || !flushQueue.isEmpty()) { try { - Map workerMap = flushQueue.poll(1, TimeUnit.SECONDS); + Map workerMap = flushQueue.poll(100, TimeUnit.MILLISECONDS); if (workerMap == null) { continue; // Timeout, check running flag } - if (workerMap == SHUTDOWN_SIGNAL) { - LOG.info("{}: Received shutdown signal", taskName); - break; - } - // Process this batch LOG.debug("{}: Background thread processing batch with {} entries", taskName, workerMap.size()); flushWithPropagation(workerMap); LOG.debug("{}: Background thread finished batch", taskName); } catch (InterruptedException e) { + // If we're stopping, ignore interrupts and keep draining the queue. + // Otherwise, preserve interrupt and exit. + if (state.get() == FlushState.STOPPING) { + LOG.debug("{}: Flusher thread interrupted while stopping; continuing to drain queue", + taskName); + Thread.interrupted(); // clear interrupt flag + continue; + } LOG.info("{}: Flusher thread interrupted", taskName); Thread.currentThread().interrupt(); break; @@ -100,6 +107,7 @@ private void flushLoop() { // Continue processing other batches } } + state.set(FlushState.STOPPED); LOG.info("{}: Async flusher stopped", taskName); } @@ -120,7 +128,7 @@ private void flushWithPropagation(Map workerMap) throws IOExcep if (actualParent == null) { actualParent = reconNamespaceSummaryManager.getNSSummary(immediateParentId); } - + if (actualParent == null) { // Parent doesn't exist in DB yet - use delta as base (has metadata like dirName, parentId) actualParent = delta; @@ -129,7 +137,7 @@ private void flushWithPropagation(Map workerMap) throws IOExcep actualParent.setNumOfFiles(actualParent.getNumOfFiles() + delta.getNumOfFiles()); actualParent.setSizeOfFiles(actualParent.getSizeOfFiles() + delta.getSizeOfFiles()); actualParent.setReplicatedSizeOfFiles(actualParent.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); - + // Merge file size buckets int[] actualBucket = actualParent.getFileSizeBucket(); int[] deltaBucket = delta.getFileSizeBucket(); @@ -144,9 +152,11 @@ private void flushWithPropagation(Map workerMap) throws IOExcep // Store updated ACTUAL parent in merged map mergedMap.put(immediateParentId, actualParent); - - // Propagate delta to ancestors (grandparent, great-grandparent, etc.) - propagateDeltaToAncestors(actualParent.getParentId(), delta, mergedMap); + + if (delta.getSizeOfFiles() > 0 || delta.getNumOfFiles() > 0) { + // Propagate delta to ancestors (grandparent, great-grandparent, etc.) + propagateDeltaToAncestors(actualParent.getParentId(), delta, mergedMap); + } } // Write merged map to DB (simple batch write, no more R-M-W needed) @@ -155,7 +165,7 @@ private void flushWithPropagation(Map workerMap) throws IOExcep } /** - * Recursively propagate delta values up the ancestor chain. + * Recursively propagate delta values up the ancestor Id. * Pattern: Check merged map first (for updates in this batch), then DB, for ACTUAL value. */ private void propagateDeltaToAncestors(long ancestorId, NSSummary delta, @@ -170,8 +180,8 @@ private void propagateDeltaToAncestors(long ancestorId, NSSummary delta, if (actualAncestor == null) { actualAncestor = reconNamespaceSummaryManager.getNSSummary(ancestorId); if (actualAncestor == null) { - // Ancestor not in DB yet, create new - actualAncestor = new NSSummary(); + // Ancestor not in DB yet return + return; } } @@ -208,51 +218,10 @@ private void writeToDb(Map mergedMap) throws IOException { @Override public void close() throws IOException { LOG.info("{}: Shutting down async flusher", taskName); - - // Wait for queue to drain before shutting down - long waitStart = System.currentTimeMillis(); - while (!flushQueue.isEmpty() && (System.currentTimeMillis() - waitStart) < 10000) { - try { - Thread.sleep(100); // Give background thread time to process - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - break; - } - } - - running.set(false); - try { - // Send shutdown signal - flushQueue.offer(SHUTDOWN_SIGNAL, 5, TimeUnit.SECONDS); - - // Wait for background thread to finish - backgroundFlusher.join(30000); // 30 second timeout - - if (backgroundFlusher.isAlive()) { - LOG.warn("{}: Background flusher did not stop gracefully, interrupting", taskName); - backgroundFlusher.interrupt(); - } - - // Flush any remaining items - int remainingCount = flushQueue.size(); - LOG.info("{}: Draining remaining {} items from queue", taskName, remainingCount); - Map remaining; - int processed = 0; - while ((remaining = flushQueue.poll()) != null) { - if (remaining != SHUTDOWN_SIGNAL) { - try { - LOG.info("{}: Draining item {} with {} entries", taskName, ++processed, remaining.size()); - flushWithPropagation(remaining); - LOG.info("{}: Successfully flushed item {}", taskName, processed); - } catch (Exception e) { - LOG.error("{}: Error flushing item {} during drain", taskName, processed, e); - // Continue draining other items - } - } - } - LOG.info("{}: Finished draining, processed {} of {} items", taskName, processed, remainingCount); - + // Tell the background thread to stop once the queue is drained. + state.set(FlushState.STOPPING); + backgroundFlusher.join(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException("Interrupted while shutting down async flusher", e); 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 94ef14ee63c3..18789eb59d2e 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 @@ -189,9 +189,6 @@ protected void handlePutKeyEventReprocess(OmKeyInfo keyInfo, Map 0 || existingNumOfFiles > 0)) { - parentNSSummary.setNumOfFiles(parentNSSummary.getNumOfFiles() + existingNumOfFiles); - parentNSSummary.setSizeOfFiles(parentNSSummary.getSizeOfFiles() + existingSizeOfFiles); - - long parentReplSize = parentNSSummary.getReplicatedSizeOfFiles(); - if (parentReplSize < 0) { - parentReplSize = 0; - } - parentNSSummary.setReplicatedSizeOfFiles(parentReplSize + existingReplicatedSizeOfFiles); - nsSummaryMap.put(parentObjectId, parentNSSummary); - - // Propagate to grandparents and beyond using reprocess-specific method - propagateSizeUpwardsReprocess(parentObjectId, existingSizeOfFiles, - existingReplicatedSizeOfFiles, existingNumOfFiles, nsSummaryMap); - } else { - nsSummaryMap.put(parentObjectId, parentNSSummary); - } + nsSummaryMap.put(parentObjectId, parentNSSummary); } protected void handleDeleteKeyEvent(OmKeyInfo keyInfo, 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 5b337e9f3cef..6df12b8eac26 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 @@ -245,7 +245,7 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { if (!processDirTableInParallel(omMetadataManager, dirFlusher)) { return false; } - // dirFlusher.close() drains queue and persists skeletons before files stage + // dirFlusher.close() drains queue and stops thread } catch (Exception ex) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB (dir phase).", ex); return false; @@ -260,12 +260,12 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { // fileFlusher.close() drains queue and persists file deltas with propagation } catch (Exception ex) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB (file phase).", ex); - return false; - } + return false; + } LOG.info("Completed a reprocess run of NSSummaryTaskWithFSO"); return true; - } + } /** * Process dirTable in parallel using per-worker maps with async flushing. @@ -309,7 +309,7 @@ private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, LOG.error("Unable to process dirTable in parallel", ex); return false; } - + // Submit any remaining worker maps for (Map remainingMap : allWorkerMaps.values()) { if (!remainingMap.isEmpty()) { From b3d00d91e69980cc817848ca1f875d75549a7f90 Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 18 Dec 2025 19:17:32 +0530 Subject: [PATCH 07/20] Removed commented 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 e9b5afe6685b..d4edd2894f9d 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 @@ -113,7 +113,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; @@ -711,152 +711,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 15c36c6a9ebb7bcee0830dc5d38816acffab5c78 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 19 Dec 2025 01:14:32 +0530 Subject: [PATCH 08/20] Fixed the size and count bug --- .../recon/tasks/NSSummaryAsyncFlusher.java | 69 +++++++++++-------- .../tasks/NSSummaryTaskDbEventHandler.java | 61 ++-------------- .../recon/tasks/NSSummaryTaskWithFSO.java | 36 +++++----- .../util/ParallelTableIteratorOperation.java | 27 ++++---- 4 files changed, 76 insertions(+), 117 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index db639c5f05f3..14918427b8ce 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -25,6 +25,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.ozone.recon.api.types.NSSummary; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; @@ -79,10 +80,11 @@ public void submitForFlush(Map workerMap) throws InterruptedExc private void flushLoop() { while (state.get() == FlushState.RUNNING || !flushQueue.isEmpty()) { try { + // Attempt to retrieve one batch from the queue, If queue empty → waits 100ms, then returns null Map workerMap = flushQueue.poll(100, TimeUnit.MILLISECONDS); if (workerMap == null) { - continue; // Timeout, check running flag + continue; } // Process this batch @@ -110,56 +112,65 @@ private void flushLoop() { state.set(FlushState.STOPPED); LOG.info("{}: Async flusher stopped", taskName); } - + /** * Flush worker map with propagation to ancestors. */ private void flushWithPropagation(Map workerMap) throws IOException { - LOG.debug("{}: Flush starting with {} entries", taskName, workerMap.size()); + Map mergedMap = new HashMap<>(); - - // For each immediate parent in worker map + + // For each object in worker map (could be either a directory or bucket) for (Map.Entry entry : workerMap.entrySet()) { - long immediateParentId = entry.getKey(); + long currentObjectId = entry.getKey(); // ← RENAMED from immediateParentId NSSummary delta = entry.getValue(); - - // Get actual parent (check merged map first, then DB) - NSSummary actualParent = mergedMap.get(immediateParentId); - if (actualParent == null) { - actualParent = reconNamespaceSummaryManager.getNSSummary(immediateParentId); + + // Get actual UpToDate nssummary (check merged map first, then DB) + NSSummary existingNSSummary = mergedMap.get(currentObjectId); + if (existingNSSummary == null) { + existingNSSummary = reconNamespaceSummaryManager.getNSSummary(currentObjectId); } - if (actualParent == null) { - // Parent doesn't exist in DB yet - use delta as base (has metadata like dirName, parentId) - actualParent = delta; + if (existingNSSummary == null) { + // Object doesn't exist in DB yet - use delta as base (has metadata like dirName, parentId) + existingNSSummary = delta; } else { - // Parent exists in DB - merge delta into it - actualParent.setNumOfFiles(actualParent.getNumOfFiles() + delta.getNumOfFiles()); - actualParent.setSizeOfFiles(actualParent.getSizeOfFiles() + delta.getSizeOfFiles()); - actualParent.setReplicatedSizeOfFiles(actualParent.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); + // Object exists in DB - merge delta into it + existingNSSummary.setNumOfFiles(existingNSSummary.getNumOfFiles() + delta.getNumOfFiles()); + existingNSSummary.setSizeOfFiles(existingNSSummary.getSizeOfFiles() + delta.getSizeOfFiles()); + existingNSSummary.setReplicatedSizeOfFiles(existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); // Merge file size buckets - int[] actualBucket = actualParent.getFileSizeBucket(); + // Question - Should we remove this? discuss with sumit + int[] actualBucket = existingNSSummary.getFileSizeBucket(); int[] deltaBucket = delta.getFileSizeBucket(); for (int i = 0; i < actualBucket.length; i++) { actualBucket[i] += deltaBucket[i]; } - actualParent.setFileSizeBucket(actualBucket); - + // Merge child dirs - actualParent.getChildDir().addAll(delta.getChildDir()); + existingNSSummary.getChildDir().addAll(delta.getChildDir()); + + // Repair dirName if existing entry is missing it and delta has the value + if (StringUtils.isEmpty(existingNSSummary.getDirName()) && StringUtils.isNotEmpty(delta.getDirName())) { + existingNSSummary.setDirName(delta.getDirName()); + } + // Repair parentId if existing entry is missing it and delta has the value + if (existingNSSummary.getParentId() == 0 && delta.getParentId() != 0) { + existingNSSummary.setParentId(delta.getParentId()); + } } - // Store updated ACTUAL parent in merged map - mergedMap.put(immediateParentId, actualParent); + // Store updated object in merged map + mergedMap.put(currentObjectId, existingNSSummary); if (delta.getSizeOfFiles() > 0 || delta.getNumOfFiles() > 0) { - // Propagate delta to ancestors (grandparent, great-grandparent, etc.) - propagateDeltaToAncestors(actualParent.getParentId(), delta, mergedMap); + // Propagate delta to ancestors (parent, grandparent, etc.) + propagateDeltaToAncestors(existingNSSummary.getParentId(), delta, mergedMap); // ← RENAMED } } - - // Write merged map to DB (simple batch write, no more R-M-W needed) + + // Write merged map to DB writeToDb(mergedMap); LOG.debug("{}: Flush completed, wrote {} entries", taskName, mergedMap.size()); } @@ -170,7 +181,7 @@ private void flushWithPropagation(Map workerMap) throws IOExcep */ private void propagateDeltaToAncestors(long ancestorId, NSSummary delta, Map mergedMap) throws IOException { - // Base case: reached root + // Base case: reached above bucket level if (ancestorId == 0) { return; } 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 18789eb59d2e..bb1f773de6a2 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 @@ -62,48 +62,9 @@ public ReconOMMetadataManager getReconOMMetadataManager() { private void updateNSSummariesToDB(Map nsSummaryMap, Collection objectIdsToBeDeleted) throws IOException { try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { - // Read-Modify-Write for each entry to prevent race conditions for (Map.Entry entry : nsSummaryMap.entrySet()) { try { - Long parentId = entry.getKey(); - NSSummary deltaSummary = entry.getValue(); - - // Step 1: READ existing value from DB - NSSummary existingSummary = reconNamespaceSummaryManager.getNSSummary(parentId); - - if (existingSummary == null) { - // First time - just write the delta as-is - existingSummary = deltaSummary; - } else { - // Step 2: MODIFY - merge delta into existing - existingSummary.setNumOfFiles(existingSummary.getNumOfFiles() + deltaSummary.getNumOfFiles()); - existingSummary.setSizeOfFiles(existingSummary.getSizeOfFiles() + deltaSummary.getSizeOfFiles()); - existingSummary.setReplicatedSizeOfFiles( - existingSummary.getReplicatedSizeOfFiles() + deltaSummary.getReplicatedSizeOfFiles()); - - // Merge file size buckets - int[] existingBucket = existingSummary.getFileSizeBucket(); - int[] deltaBucket = deltaSummary.getFileSizeBucket(); - for (int i = 0; i < existingBucket.length; i++) { - existingBucket[i] += deltaBucket[i]; - } - existingSummary.setFileSizeBucket(existingBucket); - - // Merge child directory sets - existingSummary.getChildDir().addAll(deltaSummary.getChildDir()); - - // Preserve metadata if not set in existing - if ((existingSummary.getDirName() == null || existingSummary.getDirName().isEmpty()) && - (deltaSummary.getDirName() != null && !deltaSummary.getDirName().isEmpty())) { - existingSummary.setDirName(deltaSummary.getDirName()); - } - if (existingSummary.getParentId() == 0 && deltaSummary.getParentId() != 0) { - existingSummary.setParentId(deltaSummary.getParentId()); - } - } - - // Step 3: WRITE merged result back to DB - reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, parentId, existingSummary); + reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, entry.getKey(), entry.getValue()); } catch (IOException e) { LOG.error("Unable to write Namespace Summary data in Recon DB.", e); throw e; @@ -125,14 +86,15 @@ private void updateNSSummariesToDB(Map nsSummaryMap, Collection protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); - // Try to get from map + // Try to get the NSSummary from our local map that maps NSSummaries to IDs NSSummary nsSummary = nsSummaryMap.get(parentObjectId); if (nsSummary == null) { - // Read from DB during delta updates (process method) + // If we don't have it in this batch we try to get it from the DB nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId); } if (nsSummary == null) { - // Create new instance if not found + // 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(); @@ -165,7 +127,7 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); - // Get from map only, no DB reads during reprocess + // Get from local thread map only, no DB reads during reprocess NSSummary nsSummary = nsSummaryMap.get(parentObjectId); if (nsSummary == null) { // Create new instance if not found @@ -456,15 +418,4 @@ protected void propagateSizeUpwards(long objectId, long sizeChange, long replica } } - /** - * Propagates size and count changes upwards through the parent chain during reprocess. - * Used by reprocess method - does NOT read from DB, only works with in-memory map. - * This method will be implemented to handle the specific requirements of reprocess operation. - */ - protected void propagateSizeUpwardsReprocess(long objectId, long sizeChange, long replicatedSizeChange, - int countChange, Map nsSummaryMap) - throws IOException { - // TODO: Implementation will be added to handle reprocess-specific propagation logic - } - } 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 6df12b8eac26..796d1a4beef5 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 @@ -237,7 +237,7 @@ private void handleUpdateOnFileTable(OMDBUpdateEvent workerMap = allWorkerMaps.computeIfAbsent(threadId, k -> new HashMap<>()); - + try { // Update immediate parent only, NO DB reads during reprocess handlePutDirEventReprocess(kv.getValue(), workerMap); - + // Submit to async queue when threshold reached if (workerMap.size() >= perWorkerThreshold) { asyncFlusher.submitForFlush(workerMap); @@ -317,10 +315,10 @@ private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, asyncFlusher.submitForFlush(remainingMap); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - return false; - } - } + return false; } + } + } return true; } @@ -331,22 +329,22 @@ private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, NSSummaryAsyncFlusher asyncFlusher) { Table fileTable = omMetadataManager.getFileTable(); - + // Per-worker maps for lockless updates Map> allWorkerMaps = new ConcurrentHashMap<>(); - + // Divide threshold by worker count final long perWorkerThreshold = Math.max(1, nsSummaryFlushToDBMaxThreshold / maxWorkers); - + Function, Void> kvOperation = kv -> { // Get this worker's private map long threadId = Thread.currentThread().getId(); Map workerMap = allWorkerMaps.computeIfAbsent(threadId, k -> new HashMap<>()); - + try { // Update immediate parent only, NO DB reads during reprocess handlePutKeyEventReprocess(kv.getValue(), workerMap); - + // Submit to async queue when threshold reached if (workerMap.size() >= perWorkerThreshold) { asyncFlusher.submitForFlush(workerMap); @@ -358,16 +356,16 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, } return null; }; - - try (ParallelTableIteratorOperation parallelIter = - new ParallelTableIteratorOperation<>(omMetadataManager, fileTable, StringCodec.get(), - maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + + try (ParallelTableIteratorOperation parallelIter = + new ParallelTableIteratorOperation<>(omMetadataManager, fileTable, StringCodec.get(), + maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { parallelIter.performTaskOnTableVals("NSSummaryTaskWithFSO-fileTable", null, null, kvOperation); } catch (Exception ex) { LOG.error("Unable to process fileTable in parallel", ex); return false; } - + // Submit any remaining worker maps for (Map remainingMap : allWorkerMaps.values()) { if (!remainingMap.isEmpty()) { @@ -375,7 +373,7 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, asyncFlusher.submitForFlush(remainingMap); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - return false; + return false; } } } 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 9519394482d0..60e71c6f17f9 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 @@ -55,8 +55,8 @@ public class ParallelTableIteratorOperation, V> implemen private final Codec keyCodec; // Thread Pools - private final ExecutorService iteratorExecutor; // 5 - private final ExecutorService valueExecutors; // 20 + private final ExecutorService iteratorExecutor; + private final ExecutorService valueExecutors; private final int maxNumberOfVals; private final OMMetadataManager metadataManager; @@ -72,19 +72,19 @@ public ParallelTableIteratorOperation(OMMetadataManager metadataManager, Table()); - // Create team of 20 worker threads with UNLIMITED queue + // Create team of worker threads with UNLIMITED queue this.valueExecutors = new ThreadPoolExecutor(workerCount, workerCount, 1, TimeUnit.MINUTES, new LinkedBlockingQueue<>()); - // Calculate batch size per worker (e.g., 2000 / 20 = 100 keys per batch per worker) + // Calculate batch size per worker this.maxNumberOfVals = Math.max(10, maxNumberOfValsInMemory / (workerCount)); this.logCountThreshold = logThreshold; } @@ -163,10 +163,10 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, // ===== PARALLEL PROCESSING SETUP ===== - // Queue to track iterator threads (5 threads creating work) + // Queue to track iterator threads Queue> iterFutures = new LinkedList<>(); - // Queue to track worker threads (20 threads doing work) + // Queue to track worker threads Queue> workerFutures = new ConcurrentLinkedQueue<>(); AtomicLong keyCounter = new AtomicLong(); @@ -214,7 +214,7 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, break; } - // If batch is full (2000 keys), stop collecting + // If batch is full, stop collecting if (keyValues.size() >= maxNumberOfVals) { break; } @@ -235,7 +235,7 @@ public void performTaskOnTableVals(String taskName, K startKey, K endKey, 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); } } @@ -264,12 +264,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 + // Wait for all iterator threads to finish reading waitForQueueSize(iterFutures, 0); - // Wait for all 20 worker threads to finish processing + // Wait for all worker threads to finish processing waitForQueueSize(workerFutures, 0); - // Log final stats LOG.info("{}: Parallel iteration completed - Total keys processed: {}", taskName, keyCounter.get()); } From 6402dfcb2b91b9080e1d0afccf9557cb74b8e8a3 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 19 Dec 2025 11:43:48 +0530 Subject: [PATCH 09/20] Removed comments --- .../hadoop/ozone/recon/ReconControllerModule.java | 10 +++++----- .../ozone/recon/tasks/NSSummaryAsyncFlusher.java | 7 +++---- 2 files changed, 8 insertions(+), 9 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 cd506852e0c5..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,11 +139,11 @@ 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(OmTableInsightTask.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); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 14918427b8ce..57fa0556618d 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -80,7 +80,7 @@ public void submitForFlush(Map workerMap) throws InterruptedExc private void flushLoop() { while (state.get() == FlushState.RUNNING || !flushQueue.isEmpty()) { try { - // Attempt to retrieve one batch from the queue, If queue empty → waits 100ms, then returns null + // Attempt to retrieve one batch from the queue Map workerMap = flushQueue.poll(100, TimeUnit.MILLISECONDS); if (workerMap == null) { @@ -122,7 +122,7 @@ private void flushWithPropagation(Map workerMap) throws IOExcep // For each object in worker map (could be either a directory or bucket) for (Map.Entry entry : workerMap.entrySet()) { - long currentObjectId = entry.getKey(); // ← RENAMED from immediateParentId + long currentObjectId = entry.getKey(); NSSummary delta = entry.getValue(); // Get actual UpToDate nssummary (check merged map first, then DB) @@ -141,7 +141,6 @@ private void flushWithPropagation(Map workerMap) throws IOExcep existingNSSummary.setReplicatedSizeOfFiles(existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); // Merge file size buckets - // Question - Should we remove this? discuss with sumit int[] actualBucket = existingNSSummary.getFileSizeBucket(); int[] deltaBucket = delta.getFileSizeBucket(); for (int i = 0; i < actualBucket.length; i++) { @@ -166,7 +165,7 @@ private void flushWithPropagation(Map workerMap) throws IOExcep if (delta.getSizeOfFiles() > 0 || delta.getNumOfFiles() > 0) { // Propagate delta to ancestors (parent, grandparent, etc.) - propagateDeltaToAncestors(existingNSSummary.getParentId(), delta, mergedMap); // ← RENAMED + propagateDeltaToAncestors(existingNSSummary.getParentId(), delta, mergedMap); } } From f72b9246bf42869aa6a30f5ee006491cedd4dbb5 Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 1 Jan 2026 16:31:00 +0530 Subject: [PATCH 10/20] Added parallelization for nssummarytaskWithOBS --- .../recon/tasks/NSSummaryAsyncFlusher.java | 24 ++- .../ozone/recon/tasks/NSSummaryTask.java | 2 +- .../tasks/NSSummaryTaskDbEventHandler.java | 6 + .../recon/tasks/NSSummaryTaskWithLegacy.java | 6 +- .../recon/tasks/NSSummaryTaskWithOBS.java | 139 ++++++++++++------ ...TestNSSummaryEndpointWithOBSAndLegacy.java | 2 +- .../recon/api/TestOmDBInsightEndPoint.java | 2 +- .../recon/tasks/TestNSSummaryTaskWithOBS.java | 3 +- 8 files changed, 131 insertions(+), 53 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 57fa0556618d..bb43fe17c9df 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -117,25 +117,35 @@ private void flushLoop() { * Flush worker map with propagation to ancestors. */ private void flushWithPropagation(Map workerMap) throws IOException { - + LOG.info("{}: Flush starting with {} entries", taskName, workerMap.size()); Map mergedMap = new HashMap<>(); // For each object in worker map (could be either a directory or bucket) for (Map.Entry entry : workerMap.entrySet()) { long currentObjectId = entry.getKey(); NSSummary delta = entry.getValue(); + LOG.info("Processing flush entry: {} -> files={}, size={}", currentObjectId, delta.getNumOfFiles(), delta.getSizeOfFiles()); // Get actual UpToDate nssummary (check merged map first, then DB) NSSummary existingNSSummary = mergedMap.get(currentObjectId); if (existingNSSummary == null) { existingNSSummary = reconNamespaceSummaryManager.getNSSummary(currentObjectId); + if (existingNSSummary != null) { + LOG.info("Found existing NSSummary in DB for {}: files={}, size={}", currentObjectId, existingNSSummary.getNumOfFiles(), existingNSSummary.getSizeOfFiles()); + } + } else { + LOG.info("Found existing NSSummary in mergedMap for {}: files={}, size={}", currentObjectId, existingNSSummary.getNumOfFiles(), existingNSSummary.getSizeOfFiles()); } if (existingNSSummary == null) { - // Object doesn't exist in DB yet - use delta as base (has metadata like dirName, parentId) + // Object doesn't exist in DB yet - use delta as base existingNSSummary = delta; + LOG.info("No existing NSSummary for {}, using delta as base", currentObjectId); } else { // Object exists in DB - merge delta into it + long oldFiles = existingNSSummary.getNumOfFiles(); + long oldSize = existingNSSummary.getSizeOfFiles(); + existingNSSummary.setNumOfFiles(existingNSSummary.getNumOfFiles() + delta.getNumOfFiles()); existingNSSummary.setSizeOfFiles(existingNSSummary.getSizeOfFiles() + delta.getSizeOfFiles()); existingNSSummary.setReplicatedSizeOfFiles(existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); @@ -153,17 +163,22 @@ private void flushWithPropagation(Map workerMap) throws IOExcep // Repair dirName if existing entry is missing it and delta has the value if (StringUtils.isEmpty(existingNSSummary.getDirName()) && StringUtils.isNotEmpty(delta.getDirName())) { existingNSSummary.setDirName(delta.getDirName()); + LOG.info("Repaired dirName for {}", currentObjectId); } // Repair parentId if existing entry is missing it and delta has the value if (existingNSSummary.getParentId() == 0 && delta.getParentId() != 0) { existingNSSummary.setParentId(delta.getParentId()); + LOG.info("Repaired parentId for {}", currentObjectId); } + + LOG.info("Merged NSSummary for {}: files {}->{}, size {}->{}", currentObjectId, oldFiles, existingNSSummary.getNumOfFiles(), oldSize, existingNSSummary.getSizeOfFiles()); } // Store updated object in merged map mergedMap.put(currentObjectId, existingNSSummary); if (delta.getSizeOfFiles() > 0 || delta.getNumOfFiles() > 0) { + LOG.info("Propagating delta from {} to parent {}", currentObjectId, existingNSSummary.getParentId()); // Propagate delta to ancestors (parent, grandparent, etc.) propagateDeltaToAncestors(existingNSSummary.getParentId(), delta, mergedMap); } @@ -171,9 +186,10 @@ private void flushWithPropagation(Map workerMap) throws IOExcep // Write merged map to DB writeToDb(mergedMap); - LOG.debug("{}: Flush completed, wrote {} entries", taskName, mergedMap.size()); + LOG.info("{}: Flush completed, wrote {} entries", taskName, mergedMap.size()); } - + + /** * Recursively propagate delta values up the ancestor Id. * Pattern: Check merged map first (for updates in this batch), then DB, for ACTUAL value. 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 27f4f9da062f..9d6d6f4079b0 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 @@ -124,7 +124,7 @@ public NSSummaryTask(ReconNamespaceSummaryManager new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( - reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold); + reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, 5, 20, 2000); } @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 bb1f773de6a2..c7ed2082118c 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 @@ -127,11 +127,14 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); + LOG.info("Processing key: {} (parent: {}), size: {}", keyInfo.getKeyName(), parentObjectId, keyInfo.getDataSize()); + // Get from local thread map only, no DB reads during reprocess NSSummary nsSummary = nsSummaryMap.get(parentObjectId); if (nsSummary == null) { // Create new instance if not found nsSummary = new NSSummary(); + LOG.info("Created new NSSummary for parent: {}", parentObjectId); } int[] fileBucket = nsSummary.getFileSizeBucket(); @@ -151,6 +154,9 @@ protected void handlePutKeyEventReprocess(OmKeyInfo keyInfo, Map= nsSummaryFlushToDBMaxThreshold) { if (!flushAndCommitNSToDB(nsSummaryMap)) { 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 de5ea28daac7..02e51ee2a448 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 @@ -20,11 +20,15 @@ import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.KEY_TABLE; 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.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; @@ -50,66 +55,118 @@ public class NSSummaryTaskWithOBS extends NSSummaryTaskDbEventHandler { private final long nsSummaryFlushToDBMaxThreshold; + private final int maxIterators; + private final int maxWorkers; + private final int maxKeysInMemory; + 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<>(); + // Create async flusher with queue capacity based on worker count + int queueCapacity = maxWorkers + 10; - 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 (NSSummaryAsyncFlusher asyncFlusher = + new NSSummaryAsyncFlusher(getReconNamespaceSummaryManager(), + "NSSummaryTaskWithOBS", queueCapacity)) { - setKeyParentID(keyInfo); + if (!processKeyTableInParallel(omMetadataManager, asyncFlusher)) { + return false; + } + + } catch (Exception ex) { + LOG.error("Unable to reprocess Namespace Summary data in Recon DB.", ex); + return false; + } + + LOG.info("Completed a reprocess run of NSSummaryTaskWithOBS"); + return true; + } + + private boolean processKeyTableInParallel(OMMetadataManager omMetadataManager, + NSSummaryAsyncFlusher asyncFlusher) { + Table keyTable = + omMetadataManager.getKeyTable(BUCKET_LAYOUT); + + // Per-worker maps for lockless updates + Map> allWorkerMaps = new ConcurrentHashMap<>(); + + // Divide threshold by worker count + final long perWorkerThreshold = Math.max(1, nsSummaryFlushToDBMaxThreshold / maxWorkers); - handlePutKeyEventReprocess(keyInfo, nsSummaryMap); - if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { - if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; - } + Function, Void> kvOperation = kv -> { + // Get this worker's private map + long threadId = Thread.currentThread().getId(); + Map workerMap = allWorkerMaps.computeIfAbsent(threadId, k -> new HashMap<>()); + + 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 != null && omBucketInfo.getBucketLayout() == BUCKET_LAYOUT) { + setKeyParentID(keyInfo); + // Use reprocess-specific method (no DB reads) + handlePutKeyEventReprocess(keyInfo, workerMap); + + // Submit to async queue when threshold reached + if (workerMap.size() >= perWorkerThreshold) { + asyncFlusher.submitForFlush(workerMap); + // Get fresh map for this worker + allWorkerMaps.put(threadId, new HashMap<>()); } } + } catch (IOException e) { + throw new UncheckedIOException(e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); } - } catch (IOException ioEx) { - LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", - ioEx); - nsSummaryMap.clear(); + return null; + }; + + try (ParallelTableIteratorOperation parallelIter = + new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, StringCodec.get(), + maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { + parallelIter.performTaskOnTableVals("NSSummaryTaskWithOBS-keyTable", null, null, kvOperation); + } catch (Exception ex) { + LOG.error("Unable to process keyTable in parallel", ex); return false; } - // flush and commit left out entries at end - if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + // Submit any remaining worker maps + for (Map remainingMap : allWorkerMaps.values()) { + if (!remainingMap.isEmpty()) { + try { + asyncFlusher.submitForFlush(remainingMap); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } } - LOG.debug("Completed a reprocess run of NSSummaryTaskWithOBS"); + return true; } 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 ba190bee4c35..c8d4c533e5f9 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 @@ -369,7 +369,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, 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 28a05fb71ca3..0dc47b7c0897 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 @@ -331,7 +331,7 @@ public void setUp() throws Exception { reconOMMetadataManager, ozoneConfiguration, 10); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, - reconOMMetadataManager, 10); + reconOMMetadataManager, 10,5,20,2000); NSSummaryTaskWithFSO nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, 10,5,20,2000); 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..3b9d87133345 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 @@ -60,8 +60,7 @@ false, getBucketLayout(), OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); nSSummaryTaskWithOBS = new NSSummaryTaskWithOBS(getReconNamespaceSummaryManager(), - getReconOMMetadataManager(), - threshold); + getReconOMMetadataManager(), threshold, 5, 20, 2000); } /** From 301f8d280596ddcc64a1f9b47abba6c2e00a188a Mon Sep 17 00:00:00 2001 From: arafat Date: Thu, 1 Jan 2026 16:49:14 +0530 Subject: [PATCH 11/20] Fixed the failing tests --- .../recon/tasks/NSSummaryAsyncFlusher.java | 22 ++++--------------- .../tasks/NSSummaryTaskDbEventHandler.java | 5 ----- 2 files changed, 4 insertions(+), 23 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index bb43fe17c9df..724fe403c65f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -117,35 +117,25 @@ private void flushLoop() { * Flush worker map with propagation to ancestors. */ private void flushWithPropagation(Map workerMap) throws IOException { - LOG.info("{}: Flush starting with {} entries", taskName, workerMap.size()); + Map mergedMap = new HashMap<>(); // For each object in worker map (could be either a directory or bucket) for (Map.Entry entry : workerMap.entrySet()) { long currentObjectId = entry.getKey(); NSSummary delta = entry.getValue(); - LOG.info("Processing flush entry: {} -> files={}, size={}", currentObjectId, delta.getNumOfFiles(), delta.getSizeOfFiles()); // Get actual UpToDate nssummary (check merged map first, then DB) NSSummary existingNSSummary = mergedMap.get(currentObjectId); if (existingNSSummary == null) { existingNSSummary = reconNamespaceSummaryManager.getNSSummary(currentObjectId); - if (existingNSSummary != null) { - LOG.info("Found existing NSSummary in DB for {}: files={}, size={}", currentObjectId, existingNSSummary.getNumOfFiles(), existingNSSummary.getSizeOfFiles()); - } - } else { - LOG.info("Found existing NSSummary in mergedMap for {}: files={}, size={}", currentObjectId, existingNSSummary.getNumOfFiles(), existingNSSummary.getSizeOfFiles()); } if (existingNSSummary == null) { - // Object doesn't exist in DB yet - use delta as base + // Object doesn't exist in DB yet - use delta as base (has metadata like dirName, parentId) existingNSSummary = delta; - LOG.info("No existing NSSummary for {}, using delta as base", currentObjectId); } else { // Object exists in DB - merge delta into it - long oldFiles = existingNSSummary.getNumOfFiles(); - long oldSize = existingNSSummary.getSizeOfFiles(); - existingNSSummary.setNumOfFiles(existingNSSummary.getNumOfFiles() + delta.getNumOfFiles()); existingNSSummary.setSizeOfFiles(existingNSSummary.getSizeOfFiles() + delta.getSizeOfFiles()); existingNSSummary.setReplicatedSizeOfFiles(existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); @@ -159,26 +149,22 @@ private void flushWithPropagation(Map workerMap) throws IOExcep // Merge child dirs existingNSSummary.getChildDir().addAll(delta.getChildDir()); + existingNSSummary.setFileSizeBucket(actualBucket); // Repair dirName if existing entry is missing it and delta has the value if (StringUtils.isEmpty(existingNSSummary.getDirName()) && StringUtils.isNotEmpty(delta.getDirName())) { existingNSSummary.setDirName(delta.getDirName()); - LOG.info("Repaired dirName for {}", currentObjectId); } // Repair parentId if existing entry is missing it and delta has the value if (existingNSSummary.getParentId() == 0 && delta.getParentId() != 0) { existingNSSummary.setParentId(delta.getParentId()); - LOG.info("Repaired parentId for {}", currentObjectId); } - - LOG.info("Merged NSSummary for {}: files {}->{}, size {}->{}", currentObjectId, oldFiles, existingNSSummary.getNumOfFiles(), oldSize, existingNSSummary.getSizeOfFiles()); } // Store updated object in merged map mergedMap.put(currentObjectId, existingNSSummary); if (delta.getSizeOfFiles() > 0 || delta.getNumOfFiles() > 0) { - LOG.info("Propagating delta from {} to parent {}", currentObjectId, existingNSSummary.getParentId()); // Propagate delta to ancestors (parent, grandparent, etc.) propagateDeltaToAncestors(existingNSSummary.getParentId(), delta, mergedMap); } @@ -186,7 +172,7 @@ private void flushWithPropagation(Map workerMap) throws IOExcep // Write merged map to DB writeToDb(mergedMap); - LOG.info("{}: Flush completed, wrote {} entries", taskName, mergedMap.size()); + LOG.debug("{}: Flush completed, wrote {} entries", taskName, mergedMap.size()); } 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 c7ed2082118c..b65b764d1716 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 @@ -127,14 +127,12 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { long parentObjectId = keyInfo.getParentObjectID(); - LOG.info("Processing key: {} (parent: {}), size: {}", keyInfo.getKeyName(), parentObjectId, keyInfo.getDataSize()); // Get from local thread map only, no DB reads during reprocess NSSummary nsSummary = nsSummaryMap.get(parentObjectId); if (nsSummary == null) { // Create new instance if not found nsSummary = new NSSummary(); - LOG.info("Created new NSSummary for parent: {}", parentObjectId); } int[] fileBucket = nsSummary.getFileSizeBucket(); @@ -154,9 +152,6 @@ protected void handlePutKeyEventReprocess(OmKeyInfo keyInfo, Map Date: Fri, 2 Jan 2026 16:26:44 +0530 Subject: [PATCH 12/20] Minor changes --- .../ozone/recon/tasks/NSSummaryAsyncFlusher.java | 6 ++++-- .../hadoop/ozone/recon/tasks/NSSummaryTask.java | 11 ++++++----- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 724fe403c65f..86767b079f29 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -198,8 +198,10 @@ private void propagateDeltaToAncestors(long ancestorId, NSSummary delta, } // Apply DELTA to ACTUAL ancestor - actualAncestor.setNumOfFiles(actualAncestor.getNumOfFiles() + delta.getNumOfFiles()); - actualAncestor.setSizeOfFiles(actualAncestor.getSizeOfFiles() + delta.getSizeOfFiles()); + actualAncestor.setNumOfFiles( + actualAncestor.getNumOfFiles() + delta.getNumOfFiles()); + actualAncestor.setSizeOfFiles( + actualAncestor.getSizeOfFiles() + delta.getSizeOfFiles()); actualAncestor.setReplicatedSizeOfFiles( actualAncestor.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); 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 9d6d6f4079b0..e6b69f390633 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 @@ -117,14 +117,15 @@ public NSSummaryTask(ReconNamespaceSummaryManager 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); + this.nsSummaryTaskWithFSO = + new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, + maxIterators, maxWorkers, maxKeysInMemory); this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); - this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( - reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, 5, 20, 2000); + this.nsSummaryTaskWithOBS = + new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, reconOMMetadataManager, nsSummaryFlushToDBMaxThreshold, + maxIterators, maxWorkers, maxKeysInMemory); } @Override From 3c7ff7e714f6335650be9937a3d84851cd515f36 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 2 Jan 2026 16:34:28 +0530 Subject: [PATCH 13/20] Fixed checkstyle issues --- .../ozone/recon/tasks/NSSummaryAsyncFlusher.java | 10 ++++++---- .../hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java | 1 - .../hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java | 1 - .../hadoop/ozone/recon/tasks/OmTableInsightTask.java | 4 ++-- .../hadoop/ozone/recon/api/TestContainerEndpoint.java | 8 +++----- .../recon/api/TestNSSummaryDiskUsageOrdering.java | 2 +- .../ozone/recon/api/TestNSSummaryEndpointWithFSO.java | 3 +-- .../api/TestNSSummaryEndpointWithOBSAndLegacy.java | 2 +- .../ozone/recon/api/TestOmDBInsightEndPoint.java | 6 ++---- .../ozone/recon/api/TestOpenKeysSearchEndpoint.java | 2 +- .../ozone/recon/tasks/TestNSSummaryTaskWithFSO.java | 2 +- .../recon/tasks/TestNSSummaryTreePrecomputeValues.java | 2 +- .../ozone/recon/tasks/TestOmTableInsightTask.java | 2 +- 13 files changed, 20 insertions(+), 25 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 86767b079f29..1bad271c866e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -136,9 +136,12 @@ private void flushWithPropagation(Map workerMap) throws IOExcep existingNSSummary = delta; } else { // Object exists in DB - merge delta into it - existingNSSummary.setNumOfFiles(existingNSSummary.getNumOfFiles() + delta.getNumOfFiles()); - existingNSSummary.setSizeOfFiles(existingNSSummary.getSizeOfFiles() + delta.getSizeOfFiles()); - existingNSSummary.setReplicatedSizeOfFiles(existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); + existingNSSummary.setNumOfFiles( + existingNSSummary.getNumOfFiles() + delta.getNumOfFiles()); + existingNSSummary.setSizeOfFiles( + existingNSSummary.getSizeOfFiles() + delta.getSizeOfFiles()); + existingNSSummary.setReplicatedSizeOfFiles( + existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); // Merge file size buckets int[] actualBucket = existingNSSummary.getFileSizeBucket(); @@ -175,7 +178,6 @@ private void flushWithPropagation(Map workerMap) throws IOExcep LOG.debug("{}: Flush completed, wrote {} entries", taskName, mergedMap.size()); } - /** * Recursively propagate delta values up the ancestor Id. * Pattern: Check merged map first (for updates in this batch), then DB, for ACTUAL value. 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 796d1a4beef5..bed347225382 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; 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 02e51ee2a448..b4c685fbc413 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 @@ -30,7 +30,6 @@ 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; import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; 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 5e694fd855fe..f7e8217e2293 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 @@ -208,10 +208,10 @@ private void processTableInParallel(String tableName, OMMetadataManager omMetada try (ParallelTableIteratorOperation parallelIter = new ParallelTableIteratorOperation<>( omMetadataManager, table, StringCodec.get(), maxIterators, workerCount, maxKeysInMemory, loggingThreshold)) { - + parallelIter.performTaskOnTableVals(getTaskName(), null, null, kv -> { if (kv != null) { - count.incrementAndGet(); + count.incrementAndGet(); } return null; }); 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 98eed88d018d..bdb32680d08b 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 @@ -508,8 +508,7 @@ public void testGetKeysForContainer() throws Exception { // Set up test data for FSO keys setUpFSOData(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = - new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, reconOMMetadataManager, 10, 5, 20, 2000); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); // Reprocess the container key mapper to ensure the latest mapping is used reprocessContainerKeyMapper(); @@ -596,8 +595,7 @@ public void testGetKeysForContainerWithPrevKey() throws Exception { // Reprocess the container key mapper to ensure the latest mapping is used reprocessContainerKeyMapper(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = - new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, reconOMMetadataManager, 10, 5, 20, 2000); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); response = containerEndpoint.getKeysForContainer(20L, -1, "/0/1/2/file7"); @@ -1854,7 +1852,7 @@ public void testDuplicateFSOKeysForContainerEndpoint() throws Exception { setUpDuplicateFSOFileKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + reconOMMetadataManager, 10, 5, 20, 2000); 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 3047458423a6..86256a8a3f39 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,5,20,2000); + reconOMMetadataManager, 10, 5, 20, 2000); 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 6c510850f61c..9f9bff8152c5 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 @@ -431,8 +431,7 @@ public void setUp() throws Exception { populateVolumeThree(); setUpMultiBlockReplicatedKeys(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = - new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, reconOMMetadataManager, 10, 5, 20, 2000); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } 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 c8d4c533e5f9..af9e9f46e615 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 @@ -369,7 +369,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + reconOMMetadataManager, 10, 5, 20, 2000); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, 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 0dc47b7c0897..a0f53f3172d8 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 @@ -330,11 +330,9 @@ public void setUp() throws Exception { reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration, 10); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( - reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + reconNamespaceSummaryManager, reconOMMetadataManager, 10, 5, 20, 2000); NSSummaryTaskWithFSO nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( - reconNamespaceSummaryManager, - reconOMMetadataManager, 10,5,20,2000); + reconNamespaceSummaryManager, reconOMMetadataManager, 10, 5, 20, 2000); 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 f1cd1aa9fdb6..f8dfd43a6701 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,5,20,2000); + reconOMMetadataManager, 10, 5, 20, 2000); 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 2aed805eb4bf..133f55486cd1 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,5,20,2000); + threshold, 5, 20, 2000); } 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 a5303eff05d2..da5535c96ceb 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,5,20,2000); + threshold, 5, 20, 2000); // 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 8d48e75b1813..92ee52652107 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 @@ -154,7 +154,7 @@ private void initializeInjector() throws IOException { omTableInsightTask = new OmTableInsightTask( reconGlobalStatsManager, reconOMMetadataManager); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( - reconNamespaceSummaryManager, reconOMMetadataManager, 10,5,20,2000); + reconNamespaceSummaryManager, reconOMMetadataManager, 10, 5, 20, 2000); dslContext = getDslContext(); omTableInsightTask.setTables(omTableInsightTask.getTaskTables()); From dad1fd2aaea80dc9ab5381eaaf9a9a20caee0821 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 2 Jan 2026 16:40:53 +0530 Subject: [PATCH 14/20] Fixed findbugs issue --- .../recon/tasks/NSSummaryAsyncFlusher.java | 20 +++++++++++++++---- .../recon/tasks/NSSummaryTaskWithFSO.java | 4 ++-- .../recon/tasks/NSSummaryTaskWithOBS.java | 2 +- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 1bad271c866e..2e147d13393f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -52,17 +52,29 @@ private enum FlushState { private final ReconNamespaceSummaryManager reconNamespaceSummaryManager; private final String taskName; - public NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryManager, - String taskName, - int queueCapacity) { + private NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryManager, + String taskName, + int queueCapacity) { this.reconNamespaceSummaryManager = reconNamespaceSummaryManager; this.taskName = taskName; this.flushQueue = new LinkedBlockingQueue<>(queueCapacity); this.backgroundFlusher = new Thread(this::flushLoop, taskName + "-AsyncFlusher"); this.backgroundFlusher.setDaemon(true); - this.backgroundFlusher.start(); + } + + /** + * Factory method to create and start an async flusher. + */ + public static NSSummaryAsyncFlusher create( + ReconNamespaceSummaryManager reconNamespaceSummaryManager, + String taskName, + int queueCapacity) { + NSSummaryAsyncFlusher flusher = new NSSummaryAsyncFlusher( + reconNamespaceSummaryManager, taskName, queueCapacity); + flusher.backgroundFlusher.start(); LOG.info("{}: Started async flusher with queue capacity {}", taskName, queueCapacity); + return flusher; } /** 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 bed347225382..a557caac5f42 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 @@ -239,7 +239,7 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { final int queueCapacity = maxWorkers + 10; try (NSSummaryAsyncFlusher dirFlusher = - new NSSummaryAsyncFlusher(getReconNamespaceSummaryManager(), + NSSummaryAsyncFlusher.create(getReconNamespaceSummaryManager(), "NSSummaryTaskWithFSO-dir", queueCapacity)) { if (!processDirTableInParallel(omMetadataManager, dirFlusher)) { return false; @@ -250,7 +250,7 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { } try (NSSummaryAsyncFlusher fileFlusher = - new NSSummaryAsyncFlusher(getReconNamespaceSummaryManager(), + NSSummaryAsyncFlusher.create(getReconNamespaceSummaryManager(), "NSSummaryTaskWithFSO-file", queueCapacity)) { if (!processFileTableInParallel(omMetadataManager, fileFlusher)) { return false; 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 b4c685fbc413..3ac5a9372ff7 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 @@ -78,7 +78,7 @@ public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { int queueCapacity = maxWorkers + 10; try (NSSummaryAsyncFlusher asyncFlusher = - new NSSummaryAsyncFlusher(getReconNamespaceSummaryManager(), + NSSummaryAsyncFlusher.create(getReconNamespaceSummaryManager(), "NSSummaryTaskWithOBS", queueCapacity)) { if (!processKeyTableInParallel(omMetadataManager, asyncFlusher)) { From 853546d494a5a81675d99688dcd607e18c0b3dee Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 2 Jan 2026 16:42:34 +0530 Subject: [PATCH 15/20] Refactored variable declaration code --- .../ozone/recon/tasks/NSSummaryAsyncFlusher.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 2e147d13393f..0fdba095b7c9 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -39,18 +39,18 @@ public class NSSummaryAsyncFlusher implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(NSSummaryAsyncFlusher.class); - private enum FlushState { - RUNNING, - STOPPING, - STOPPED - } - private final BlockingQueue> flushQueue; private final Thread backgroundFlusher; private final AtomicReference state = new AtomicReference<>(FlushState.RUNNING); private final ReconNamespaceSummaryManager reconNamespaceSummaryManager; private final String taskName; + + private enum FlushState { + RUNNING, + STOPPING, + STOPPED + } private NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryManager, String taskName, From 9b56da2ab34a58971e075d169270ae63c1bd4ca9 Mon Sep 17 00:00:00 2001 From: arafat Date: Fri, 2 Jan 2026 17:14:30 +0530 Subject: [PATCH 16/20] Removed unused code --- .../recon/tasks/NSSummaryAsyncFlusher.java | 2 +- .../recon/tasks/NSSummaryTaskWithFSO.java | 50 ------------------- 2 files changed, 1 insertion(+), 51 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index 0fdba095b7c9..cddc59a3caa5 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -36,7 +36,7 @@ * Async flusher for NSSummary maps with background thread. * Workers submit their maps to a queue, background thread processes them. */ -public class NSSummaryAsyncFlusher implements Closeable { +public final class NSSummaryAsyncFlusher implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(NSSummaryAsyncFlusher.class); private final BlockingQueue> flushQueue; 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 a557caac5f42..6db6e355c8c5 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 @@ -379,54 +379,4 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, return true; } - /** - * Merge collection of worker maps into a single new map. - */ - private Map mergeWorkerMaps(Collection> workerMaps) { - Map mergedMap = new HashMap<>(); - mergeWorkerMapsIntoBase(workerMaps, mergedMap); - return mergedMap; - } - - /** - * Merge worker maps into base map, combining NSSummary values. - */ - private void mergeWorkerMapsIntoBase(Collection> workerMaps, - Map baseMap) { - for (Map workerMap : workerMaps) { - for (Map.Entry entry : workerMap.entrySet()) { - Long parentId = entry.getKey(); - NSSummary workerSummary = entry.getValue(); - - // Get or create in base map - NSSummary baseSummary = baseMap.computeIfAbsent(parentId, k -> new NSSummary()); - - // Merge worker's data into base - baseSummary.setNumOfFiles(baseSummary.getNumOfFiles() + workerSummary.getNumOfFiles()); - baseSummary.setSizeOfFiles(baseSummary.getSizeOfFiles() + workerSummary.getSizeOfFiles()); - baseSummary.setReplicatedSizeOfFiles( - baseSummary.getReplicatedSizeOfFiles() + workerSummary.getReplicatedSizeOfFiles()); - - // Merge file size buckets - int[] baseBucket = baseSummary.getFileSizeBucket(); - int[] workerBucket = workerSummary.getFileSizeBucket(); - for (int i = 0; i < baseBucket.length; i++) { - baseBucket[i] += workerBucket[i]; - } - baseSummary.setFileSizeBucket(baseBucket); - - // Merge child directory sets - baseSummary.getChildDir().addAll(workerSummary.getChildDir()); - - // Note: dirName and parentId should be consistent across workers for same ID - if ((baseSummary.getDirName() == null || baseSummary.getDirName().isEmpty()) && - (workerSummary.getDirName() != null && !workerSummary.getDirName().isEmpty())) { - baseSummary.setDirName(workerSummary.getDirName()); - } - if (baseSummary.getParentId() == 0 && workerSummary.getParentId() != 0) { - baseSummary.setParentId(workerSummary.getParentId()); - } - } - } - } } From ee181a99dc56b8081457492a8275fa6ddd571c72 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 7 Jan 2026 12:58:29 +0530 Subject: [PATCH 17/20] Fixed the directory iteration performance --- .../ozone/recon/ReconServerConfigKeys.java | 2 +- .../recon/tasks/NSSummaryAsyncFlusher.java | 30 +++++++++++-------- .../recon/tasks/NSSummaryTaskWithFSO.java | 14 ++++++++- .../recon/tasks/NSSummaryTaskWithOBS.java | 8 ++++- .../util/ParallelTableIteratorOperation.java | 2 +- 5 files changed, 39 insertions(+), 17 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 ba357f4ba145..386479123a20 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 @@ -153,7 +153,7 @@ public final class ReconServerConfigKeys { "ozone.recon.nssummary.flush.db.max.threshold"; public static final long - OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT = 150 * 1000L; + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT = 150 * 1000L * 2; public static final String OZONE_RECON_CONTAINER_KEY_FLUSH_TO_DB_MAX_THRESHOLD = diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index cddc59a3caa5..f8984735191b 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -148,23 +148,27 @@ private void flushWithPropagation(Map workerMap) throws IOExcep existingNSSummary = delta; } else { // Object exists in DB - merge delta into it - existingNSSummary.setNumOfFiles( - existingNSSummary.getNumOfFiles() + delta.getNumOfFiles()); - existingNSSummary.setSizeOfFiles( - existingNSSummary.getSizeOfFiles() + delta.getSizeOfFiles()); - existingNSSummary.setReplicatedSizeOfFiles( - existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); + + // Skip numeric merging if delta has no file data (e.g., directory skeleton with zero counts) + if (delta.getNumOfFiles() > 0 || delta.getSizeOfFiles() > 0) { + existingNSSummary.setNumOfFiles( + existingNSSummary.getNumOfFiles() + delta.getNumOfFiles()); + existingNSSummary.setSizeOfFiles( + existingNSSummary.getSizeOfFiles() + delta.getSizeOfFiles()); + existingNSSummary.setReplicatedSizeOfFiles( + existingNSSummary.getReplicatedSizeOfFiles() + delta.getReplicatedSizeOfFiles()); - // Merge file size buckets - int[] actualBucket = existingNSSummary.getFileSizeBucket(); - int[] deltaBucket = delta.getFileSizeBucket(); - for (int i = 0; i < actualBucket.length; i++) { - actualBucket[i] += deltaBucket[i]; + // Merge file size buckets + int[] actualBucket = existingNSSummary.getFileSizeBucket(); + int[] deltaBucket = delta.getFileSizeBucket(); + for (int i = 0; i < actualBucket.length; i++) { + actualBucket[i] += deltaBucket[i]; + } + existingNSSummary.setFileSizeBucket(actualBucket); } - // Merge child dirs + // Merge child dirs (needed for directory relationships) existingNSSummary.getChildDir().addAll(delta.getChildDir()); - existingNSSummary.setFileSizeBucket(actualBucket); // Repair dirName if existing entry is missing it and delta has the value if (StringUtils.isEmpty(existingNSSummary.getDirName()) && StringUtils.isNotEmpty(delta.getDirName())) { 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 6db6e355c8c5..2eddef918c29 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 @@ -236,7 +236,7 @@ private void handleUpdateOnFileTable(OMDBUpdateEvent parallelIter = new ParallelTableIteratorOperation<>(omMetadataManager, dirTable, StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { @@ -306,6 +309,9 @@ private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, LOG.error("Unable to process dirTable in parallel", ex); return false; } + + long dirEndTime = System.currentTimeMillis(); + LOG.debug("Completed dirTable parallel iteration in {} ms", (dirEndTime - dirStartTime)); // Submit any remaining worker maps for (Map remainingMap : allWorkerMaps.values()) { @@ -356,6 +362,9 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, return null; }; + LOG.debug("Starting fileTable parallel iteration"); + long fileStartTime = System.currentTimeMillis(); + try (ParallelTableIteratorOperation parallelIter = new ParallelTableIteratorOperation<>(omMetadataManager, fileTable, StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { @@ -364,6 +373,9 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, LOG.error("Unable to process fileTable in parallel", ex); return false; } + + long fileEndTime = System.currentTimeMillis(); + LOG.debug("Completed fileTable parallel iteration in {} ms", (fileEndTime - fileStartTime)); // Submit any remaining worker maps for (Map remainingMap : allWorkerMaps.values()) { 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 3ac5a9372ff7..056fb5cd9ea9 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 @@ -75,7 +75,7 @@ public NSSummaryTaskWithOBS( public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { // Create async flusher with queue capacity based on worker count - int queueCapacity = maxWorkers + 10; + int queueCapacity = maxWorkers * 2; try (NSSummaryAsyncFlusher asyncFlusher = NSSummaryAsyncFlusher.create(getReconNamespaceSummaryManager(), @@ -145,6 +145,9 @@ private boolean processKeyTableInParallel(OMMetadataManager omMetadataManager, return null; }; + LOG.debug("Starting keyTable parallel iteration"); + long keyStartTime = System.currentTimeMillis(); + try (ParallelTableIteratorOperation parallelIter = new ParallelTableIteratorOperation<>(omMetadataManager, keyTable, StringCodec.get(), maxIterators, maxWorkers, maxKeysInMemory, nsSummaryFlushToDBMaxThreshold)) { @@ -153,6 +156,9 @@ private boolean processKeyTableInParallel(OMMetadataManager omMetadataManager, LOG.error("Unable to process keyTable in parallel", ex); return false; } + + long keyEndTime = System.currentTimeMillis(); + LOG.debug("Completed keyTable parallel iteration in {} ms", (keyEndTime - keyStartTime)); // Submit any remaining worker maps for (Map remainingMap : allWorkerMaps.values()) { 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 60e71c6f17f9..3c378c36cd9f 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 @@ -143,7 +143,7 @@ private void waitForQueueSize(Queue> futures, int expectedSize) public void performTaskOnTableVals(String taskName, K startKey, K endKey, Function, Void> keyOperation) throws IOException, ExecutionException, InterruptedException { List bounds = getBounds(startKey, endKey); - + LOG.debug("Length of the bounds - {}", bounds.size()); // Fallback for small tables (no SST files yet - data only in memtable) if (bounds.size() < 2) { try (TableIterator> iter = table.iterator()) { From d357492f171c4d6d77a470ae63e20442b1389cc2 Mon Sep 17 00:00:00 2001 From: arafat Date: Wed, 7 Jan 2026 13:25:46 +0530 Subject: [PATCH 18/20] Improved error handling code --- .../recon/tasks/NSSummaryAsyncFlusher.java | 64 +++++++++++++++++-- .../recon/tasks/NSSummaryTaskWithFSO.java | 12 ++++ .../recon/tasks/NSSummaryTaskWithOBS.java | 6 ++ 3 files changed, 75 insertions(+), 7 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index f8984735191b..b6988570068f 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -43,13 +43,15 @@ public final class NSSummaryAsyncFlusher implements Closeable { private final Thread backgroundFlusher; private final AtomicReference state = new AtomicReference<>(FlushState.RUNNING); + private volatile Exception failureCause = null; private final ReconNamespaceSummaryManager reconNamespaceSummaryManager; private final String taskName; private enum FlushState { RUNNING, STOPPING, - STOPPED + STOPPED, + FAILED } private NSSummaryAsyncFlusher(ReconNamespaceSummaryManager reconNamespaceSummaryManager, @@ -80,10 +82,34 @@ public static NSSummaryAsyncFlusher create( /** * Submit a worker map for async flushing. * Blocks if queue is full (natural backpressure). + * @throws IOException if the async flusher has failed */ - public void submitForFlush(Map workerMap) throws InterruptedException { + public void submitForFlush(Map workerMap) + throws InterruptedException, IOException { + // Check if flusher has failed - reject new submissions + if (state.get() == FlushState.FAILED) { + throw new IOException(taskName + ": Cannot submit - async flusher has failed", + failureCause); + } + flushQueue.put(workerMap); - LOG.debug("{}: Submitted map with {} entries, queue size now {}", taskName, workerMap.size(), flushQueue.size()); + LOG.debug("{}: Submitted map with {} entries, queue size now {}", + taskName, workerMap.size(), flushQueue.size()); + } + + /** + * Check if the async flusher has encountered any failures. + * Workers should call this periodically to detect failures fast. + * @throws IOException if a failure has occurred + */ + public void checkForFailures() throws IOException { + if (state.get() == FlushState.FAILED && failureCause != null) { + if (failureCause instanceof IOException) { + throw (IOException) failureCause; + } else { + throw new IOException(taskName + ": Async flusher failed", failureCause); + } + } } /** @@ -116,13 +142,33 @@ private void flushLoop() { LOG.info("{}: Flusher thread interrupted", taskName); Thread.currentThread().interrupt(); break; + } catch (IOException e) { + // For DB write errors + LOG.error("{}: FATAL - DB write failed, stopping async flusher. " + + "Remaining {} batches in queue will NOT be processed. " + + "Workers will be stopped immediately.", + taskName, flushQueue.size(), e); + failureCause = e; + state.set(FlushState.FAILED); + break; } catch (Exception e) { - LOG.error("{}: Error in flush loop", taskName, e); - // Continue processing other batches + // Other unexpected errors are also fatal + LOG.error("{}: FATAL - Unexpected error in flush loop, stopping async flusher. " + + "Remaining {} batches in queue will NOT be processed. " + + "Workers will be stopped immediately.", + taskName, flushQueue.size(), e); + failureCause = e; + state.set(FlushState.FAILED); + break; } } - state.set(FlushState.STOPPED); - LOG.info("{}: Async flusher stopped", taskName); + + // Only set STOPPED if we didn't fail + if (state.get() != FlushState.FAILED) { + state.set(FlushState.STOPPED); + } + + LOG.info("{}: Async flusher stopped with state: {}", taskName, state.get()); } /** @@ -254,6 +300,10 @@ public void close() throws IOException { // Tell the background thread to stop once the queue is drained. state.set(FlushState.STOPPING); backgroundFlusher.join(); + + // Check if there were any failures during processing + checkForFailures(); + } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException("Interrupted while shutting down async flusher", e); 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 2eddef918c29..eb26ad8abfa1 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 @@ -283,6 +283,9 @@ private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, Map workerMap = allWorkerMaps.computeIfAbsent(threadId, k -> new HashMap<>()); try { + // Check if async flusher has failed - stop immediately if so + asyncFlusher.checkForFailures(); + // Update immediate parent only, NO DB reads during reprocess handlePutDirEventReprocess(kv.getValue(), workerMap); @@ -321,6 +324,9 @@ private boolean processDirTableInParallel(OMMetadataManager omMetadataManager, } catch (InterruptedException e) { Thread.currentThread().interrupt(); return false; + } catch (IOException e) { + LOG.error("Failed to submit remaining map for flush", e); + return false; } } } @@ -347,6 +353,9 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, Map workerMap = allWorkerMaps.computeIfAbsent(threadId, k -> new HashMap<>()); try { + // Check if async flusher has failed - stop immediately if so + asyncFlusher.checkForFailures(); + // Update immediate parent only, NO DB reads during reprocess handlePutKeyEventReprocess(kv.getValue(), workerMap); @@ -385,6 +394,9 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, } catch (InterruptedException e) { Thread.currentThread().interrupt(); return false; + } catch (IOException e) { + LOG.error("Failed to submit remaining map for flush", e); + return false; } } } 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 056fb5cd9ea9..7ebd0c96f5f4 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 @@ -125,6 +125,9 @@ private boolean processKeyTableInParallel(OMMetadataManager omMetadataManager, .getBucketTable().getSkipCache(bucketDBKey); if (omBucketInfo != null && omBucketInfo.getBucketLayout() == BUCKET_LAYOUT) { + // Check if async flusher has failed - stop immediately if so + asyncFlusher.checkForFailures(); + setKeyParentID(keyInfo); // Use reprocess-specific method (no DB reads) handlePutKeyEventReprocess(keyInfo, workerMap); @@ -168,6 +171,9 @@ private boolean processKeyTableInParallel(OMMetadataManager omMetadataManager, } catch (InterruptedException e) { Thread.currentThread().interrupt(); return false; + } catch (IOException e) { + LOG.error("Failed to submit remaining map for flush", e); + return false; } } } From b96515f9c87e845f9aefca493de4faf4f9547baa Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 12 Jan 2026 12:44:41 +0530 Subject: [PATCH 19/20] Fixed the failing tests and build --- .../hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java | 2 +- .../ozone/recon/tasks/NSSummaryTaskDbEventHandler.java | 3 +-- .../hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java | 3 ++- .../hadoop/ozone/recon/tasks/NSSummaryTaskWithOBS.java | 6 ++++-- 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java index b6988570068f..d25eaced22ff 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryAsyncFlusher.java @@ -280,7 +280,7 @@ private void propagateDeltaToAncestors(long ancestorId, NSSummary delta, * Write merged map to DB using batch operation. */ private void writeToDb(Map mergedMap) throws IOException { - try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { + try (RDBBatchOperation rdbBatchOperation = RDBBatchOperation.newAtomicOperation()) { for (Map.Entry entry : mergedMap.entrySet()) { reconNamespaceSummaryManager.batchStoreNSSummaries( rdbBatchOperation, entry.getKey(), entry.getValue()); 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 bace3d5480d0..f4e7163e3b90 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 @@ -124,8 +124,7 @@ protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map nsSummaryMap) throws IOException { - long parentObjectId = keyInfo.getParentObjectID(); + NSSummary> nsSummaryMap, long parentObjectId) throws IOException { // Get from local thread map only, no DB reads during reprocess NSSummary nsSummary = nsSummaryMap.get(parentObjectId); 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 31c15a965276..2158326cefb2 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 @@ -357,7 +357,8 @@ private boolean processFileTableInParallel(OMMetadataManager omMetadataManager, asyncFlusher.checkForFailures(); // Update immediate parent only, NO DB reads during reprocess - handlePutKeyEventReprocess(kv.getValue(), workerMap); + OmKeyInfo keyInfo = kv.getValue(); + handlePutKeyEventReprocess(keyInfo, workerMap, keyInfo.getParentObjectID()); // Submit to async queue when threshold reached if (workerMap.size() >= perWorkerThreshold) { 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 a60d88b6ec56..a78439616729 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 @@ -128,9 +128,11 @@ private boolean processKeyTableInParallel(OMMetadataManager omMetadataManager, // Check if async flusher has failed - stop immediately if so asyncFlusher.checkForFailures(); - setKeyParentID(keyInfo); + // Look up parent ID for OBS keys (not populated in keyInfo from DB) + long parentObjectId = getKeyParentID(keyInfo); + // Use reprocess-specific method (no DB reads) - handlePutKeyEventReprocess(keyInfo, workerMap); + handlePutKeyEventReprocess(keyInfo, workerMap, parentObjectId); // Submit to async queue when threshold reached if (workerMap.size() >= perWorkerThreshold) { From 88eff5e707075c0635cfe0991f76d98930f9100b Mon Sep 17 00:00:00 2001 From: arafat Date: Mon, 12 Jan 2026 12:45:50 +0530 Subject: [PATCH 20/20] Fixed queue size --- .../apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 2158326cefb2..9f281a908eed 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 @@ -236,7 +236,7 @@ private void handleUpdateOnFileTable(OMDBUpdateEvent