From f752af8bee2d47704feb9234f6c7d1ec4c66debb Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Mon, 20 Jan 2025 12:13:33 +0530 Subject: [PATCH 01/13] HDDS-12062. Recon - Error handling in NSSummaryTask to avoid data inconsistencies. --- .../impl/OzoneManagerServiceProviderImpl.java | 2 + .../recon/tasks/ContainerKeyMapperTask.java | 17 +++-- .../ozone/recon/tasks/FileSizeCountTask.java | 8 ++- .../ozone/recon/tasks/NSSummaryTask.java | 46 ++++++------ .../tasks/NSSummaryTaskDbEventHandler.java | 12 ++-- .../recon/tasks/NSSummaryTaskWithFSO.java | 46 ++++++++---- .../recon/tasks/NSSummaryTaskWithLegacy.java | 39 +++++++--- .../recon/tasks/NSSummaryTaskWithOBS.java | 44 ++++++++---- .../ozone/recon/tasks/OmTableInsightTask.java | 8 ++- .../hadoop/ozone/recon/tasks/ReconOmTask.java | 7 +- .../recon/tasks/ReconTaskControllerImpl.java | 71 ++++++++++++++----- .../recon/api/TestContainerEndpoint.java | 4 +- .../api/TestNSSummaryDiskUsageOrdering.java | 2 +- .../api/TestNSSummaryEndpointWithFSO.java | 2 +- .../api/TestNSSummaryEndpointWithLegacy.java | 2 +- ...TestNSSummaryEndpointWithOBSAndLegacy.java | 4 +- .../recon/api/TestOmDBInsightEndPoint.java | 6 +- .../recon/api/TestOpenKeysSearchEndpoint.java | 2 +- .../ozone/recon/tasks/DummyReconDBTask.java | 7 +- .../tasks/TestContainerKeyMapperTask.java | 6 +- .../recon/tasks/TestFileSizeCountTask.java | 8 +-- .../ozone/recon/tasks/TestNSSummaryTask.java | 2 +- .../recon/tasks/TestNSSummaryTaskWithFSO.java | 7 +- .../tasks/TestNSSummaryTaskWithLegacy.java | 9 ++- .../TestNSSummaryTaskWithLegacyOBSLayout.java | 10 ++- .../recon/tasks/TestNSSummaryTaskWithOBS.java | 11 ++- .../recon/tasks/TestOmTableInsightTask.java | 27 ++++--- .../tasks/TestReconTaskControllerImpl.java | 7 +- 28 files changed, 284 insertions(+), 132 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 d5b7b1cfc911..28e179d26a77 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 @@ -529,7 +529,9 @@ boolean innerGetAndApplyDeltaUpdatesFromOM(long fromSequenceNumber, } for (byte[] data : dbUpdates.getData()) { try (ManagedWriteBatch writeBatch = new ManagedWriteBatch(data)) { + // Events gets populated in events list in OMDBUpdatesHandler with call back for put/delete/update writeBatch.iterate(omdbUpdatesHandler); + // Commit the OM DB transactions in recon rocks DB and sync here. try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation(writeBatch)) { try (ManagedWriteOptions wOpts = new ManagedWriteOptions()) { diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java index bf34c9f89301..e8379e1c18fd 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java @@ -192,8 +192,14 @@ public Collection getTaskTables() { } @Override - public Pair process(OMUpdateEventBatch events) { + public Pair> process(OMUpdateEventBatch events, + int seekPos) { Iterator eventIterator = events.getIterator(); + int itrPos = 0; + while (eventIterator.hasNext() && itrPos < seekPos) { + eventIterator.next(); + itrPos++; + } int eventCount = 0; final Collection taskTables = getTaskTables(); @@ -249,18 +255,21 @@ public Pair process(OMUpdateEventBatch events) { } catch (IOException e) { LOG.error("Unexpected exception while updating key data : {} ", updatedKey, e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), + new ImmutablePair<>(0, false)); } } try { writeToTheDB(containerKeyMap, containerKeyCountMap, deletedKeyCountList); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), + new ImmutablePair<>(0, false)); } LOG.debug("{} successfully processed {} OM DB update event(s).", getTaskName(), eventCount); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), + new ImmutablePair<>(0, true)); } private void writeToTheDB(Map containerKeyMap, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java index cbbbe6c3732e..29cc0a0aa5e7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java @@ -144,7 +144,8 @@ public Collection getTaskTables() { * @return Pair */ @Override - public Pair process(OMUpdateEventBatch events) { + public Pair> process(OMUpdateEventBatch events, + int seekPos) { Iterator eventIterator = events.getIterator(); Map fileSizeCountMap = new HashMap<>(); final Collection taskTables = getTaskTables(); @@ -190,7 +191,8 @@ public Pair process(OMUpdateEventBatch events) { } catch (Exception e) { LOG.error("Unexpected exception while processing key {}.", updatedKey, e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), + new ImmutablePair<>(0, false)); } } else { LOG.warn("Unexpected value type {} for key {}. Skipping processing.", @@ -199,7 +201,7 @@ public Pair process(OMUpdateEventBatch events) { } writeCountsToDB(false, fileSizeCountMap); LOG.debug("Completed a 'process' run of FileSizeCountTask."); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } /** diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index 256e0c687ba5..a12b75138f2a 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 @@ -40,6 +40,9 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; + /** * Task to query data from OMDB and write into Recon RocksDB. * Reprocess() will take a snapshots on OMDB, and iterate the keyTable, @@ -70,7 +73,6 @@ public class NSSummaryTask implements ReconOmTask { private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO; private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy; private final NSSummaryTaskWithOBS nsSummaryTaskWithOBS; - private final OzoneConfiguration ozoneConfiguration; @Inject public NSSummaryTask(ReconNamespaceSummaryManager @@ -81,16 +83,19 @@ public NSSummaryTask(ReconNamespaceSummaryManager ozoneConfiguration) { 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); + this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( - reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconNamespaceSummaryManager, reconOMMetadataManager, + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( - reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconNamespaceSummaryManager, reconOMMetadataManager, + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( - reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconNamespaceSummaryManager, reconOMMetadataManager, + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); } @Override @@ -99,20 +104,24 @@ public String getTaskName() { } @Override - public Pair process(OMUpdateEventBatch events) { - boolean success = nsSummaryTaskWithFSO.processWithFSO(events); - if (!success) { + public Pair> process( + OMUpdateEventBatch events, int seekPosition) { + Pair itrPosStatusPair = + nsSummaryTaskWithFSO.processWithFSO(events, seekPosition); + if (!itrPosStatusPair.getRight()) { LOG.error("processWithFSO failed."); } - success = nsSummaryTaskWithLegacy.processWithLegacy(events); - if (!success) { + itrPosStatusPair = + nsSummaryTaskWithLegacy.processWithLegacy(events, seekPosition); + if (!itrPosStatusPair.getRight()) { LOG.error("processWithLegacy failed."); } - success = nsSummaryTaskWithOBS.processWithOBS(events); - if (!success) { + itrPosStatusPair = + nsSummaryTaskWithOBS.processWithOBS(events, seekPosition); + if (!itrPosStatusPair.getRight()) { LOG.error("processWithOBS failed."); } - return new ImmutablePair<>(getTaskName(), success); + return new ImmutablePair<>(getTaskName(), itrPosStatusPair); } @Override @@ -151,12 +160,9 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { return new ImmutablePair<>(getTaskName(), false); } } - } catch (InterruptedException ex) { + } catch (InterruptedException | ExecutionException ex) { LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex); return new ImmutablePair<>(getTaskName(), false); - } catch (ExecutionException ex2) { - LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex2); - return new ImmutablePair<>(getTaskName(), false); } finally { executorService.shutdown(); 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 888ec5319f2f..060dce3bea28 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 @@ -73,15 +73,16 @@ public ReconOMMetadataManager getReconOMMetadataManager() { protected void writeNSSummariesToDB(Map nsSummaryMap) throws IOException { try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { - nsSummaryMap.keySet().forEach((Long key) -> { + for (Long key : nsSummaryMap.keySet()) { try { reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, key, nsSummaryMap.get(key)); } catch (IOException e) { LOG.error("Unable to write Namespace Summary data in Recon DB.", e); + throw e; } - }); + } reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation); } } @@ -210,15 +211,16 @@ protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo, protected boolean flushAndCommitNSToDB(Map nsSummaryMap) { try { writeNSSummariesToDB(nsSummaryMap); - nsSummaryMap.clear(); } catch (IOException e) { LOG.error("Unable to write Namespace Summary data in Recon DB.", e); return false; + } finally { + nsSummaryMap.clear(); } return true; } - protected boolean checkAndCallFlushToDB( +/* protected boolean checkAndCallFlushToDB( Map nsSummaryMap) { // if map contains more than entries, flush to DB and clear the map if (null != nsSummaryMap && nsSummaryMap.size() >= @@ -226,5 +228,5 @@ protected boolean checkAndCallFlushToDB( return flushAndCommitNSToDB(nsSummaryMap); } return true; - } + }*/ } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java index 037376bea0b6..2d49578bd08c 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 @@ -17,6 +17,8 @@ */ package org.apache.hadoop.ozone.recon.tasks; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; @@ -35,6 +37,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE; @@ -48,14 +51,18 @@ public class NSSummaryTaskWithFSO extends NSSummaryTaskDbEventHandler { private static final Logger LOG = LoggerFactory.getLogger(NSSummaryTaskWithFSO.class); + private final long nsSummaryFlushToDBMaxThreshold; + public NSSummaryTaskWithFSO(ReconNamespaceSummaryManager reconNamespaceSummaryManager, ReconOMMetadataManager reconOMMetadataManager, OzoneConfiguration - ozoneConfiguration) { + ozoneConfiguration, + long nsSummaryFlushToDBMaxThreshold) { super(reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration); + this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; } // We only listen to updates from FSO-enabled KeyTable(FileTable) and DirTable @@ -63,15 +70,23 @@ public Collection getTaskTables() { return Arrays.asList(FILE_TABLE, DIRECTORY_TABLE); } - public boolean processWithFSO(OMUpdateEventBatch events) { + public Pair processWithFSO(OMUpdateEventBatch events, + int seekPos) { Iterator eventIterator = events.getIterator(); + int itrPos = 0; + while (eventIterator.hasNext() && itrPos < seekPos) { + eventIterator.next(); + itrPos++; + } final Collection taskTables = getTaskTables(); Map nsSummaryMap = new HashMap<>(); + int eventCounter = 0; while (eventIterator.hasNext()) { OMDBUpdateEvent omdbUpdateEvent = eventIterator.next(); OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction(); + eventCounter++; // we only process updates on OM's FileTable and Dirtable String table = omdbUpdateEvent.getTable(); @@ -150,20 +165,23 @@ public boolean processWithFSO(OMUpdateEventBatch events) { } catch (IOException ioEx) { LOG.error("Unable to process Namespace Summary data in Recon DB. ", ioEx); - return false; + nsSummaryMap.clear(); + return new ImmutablePair<>(seekPos, false); } - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return new ImmutablePair<>(seekPos, false); + } + seekPos = eventCounter + 1; } } // flush and commit left out entries at end if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + return new ImmutablePair<>(seekPos, false); } - LOG.debug("Completed a process run of NSSummaryTaskWithFSO"); - return true; + return new ImmutablePair<>(seekPos, true); } public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { @@ -179,8 +197,10 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { Table.KeyValue kv = dirTableIter.next(); OmDirectoryInfo directoryInfo = kv.getValue(); handlePutDirEvent(directoryInfo, nsSummaryMap); - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; + } } } } @@ -195,8 +215,10 @@ public boolean reprocessWithFSO(OMMetadataManager omMetadataManager) { Table.KeyValue kv = keyTableIter.next(); OmKeyInfo keyInfo = kv.getValue(); handlePutKeyEvent(keyInfo, nsSummaryMap); - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; + } } } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java index 121b280bf683..13d798e19290 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 @@ -17,6 +17,8 @@ */ package org.apache.hadoop.ozone.recon.tasks; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; @@ -53,23 +55,34 @@ public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler { LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class); private boolean enableFileSystemPaths; + private final long nsSummaryFlushToDBMaxThreshold; public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager reconNamespaceSummaryManager, ReconOMMetadataManager reconOMMetadataManager, OzoneConfiguration - ozoneConfiguration) { + ozoneConfiguration, + long nsSummaryFlushToDBMaxThreshold) { super(reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration); // true if FileSystemPaths enabled enableFileSystemPaths = ozoneConfiguration .getBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS, OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT); + this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; } - public boolean processWithLegacy(OMUpdateEventBatch events) { + public Pair processWithLegacy(OMUpdateEventBatch events, + int seekPos) { Iterator eventIterator = events.getIterator(); + int itrPos = 0; + while (eventIterator.hasNext() && itrPos < seekPos) { + eventIterator.next(); + itrPos++; + } + + int eventCounter = 0; Map nsSummaryMap = new HashMap<>(); ReconOMMetadataManager metadataManager = getReconOMMetadataManager(); @@ -77,6 +90,7 @@ public boolean processWithLegacy(OMUpdateEventBatch events) { OMDBUpdateEvent omdbUpdateEvent = eventIterator.next(); OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction(); + eventCounter++; // we only process updates on OM's KeyTable String table = omdbUpdateEvent.getTable(); @@ -115,20 +129,24 @@ public boolean processWithLegacy(OMUpdateEventBatch events) { } catch (IOException ioEx) { LOG.error("Unable to process Namespace Summary data in Recon DB. ", ioEx); - return false; + nsSummaryMap.clear(); + return new ImmutablePair<>(seekPos, false); } - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return new ImmutablePair<>(seekPos, false); + } + seekPos = eventCounter + 1; } } // flush and commit left out entries at end if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + return new ImmutablePair<>(seekPos, false); } LOG.debug("Completed a process run of NSSummaryTaskWithLegacy"); - return true; + return new ImmutablePair<>(seekPos, true); } private void processWithFileSystemLayout(OmKeyInfo updatedKeyInfo, @@ -279,14 +297,17 @@ public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) { setParentBucketId(keyInfo); handlePutKeyEvent(keyInfo, nsSummaryMap); } - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; + } } } } } catch (IOException ioEx) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", ioEx); + nsSummaryMap.clear(); 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 e40725167404..927578cb8c9c 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 @@ -18,6 +18,8 @@ package org.apache.hadoop.ozone.recon.tasks; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; @@ -50,13 +52,17 @@ public class NSSummaryTaskWithOBS extends NSSummaryTaskDbEventHandler { private static final Logger LOG = LoggerFactory.getLogger(NSSummaryTaskWithOBS.class); + private final long nsSummaryFlushToDBMaxThreshold; + public NSSummaryTaskWithOBS( ReconNamespaceSummaryManager reconNamespaceSummaryManager, ReconOMMetadataManager reconOMMetadataManager, - OzoneConfiguration ozoneConfiguration) { + OzoneConfiguration ozoneConfiguration, + long nsSummaryFlushToDBMaxThreshold) { super(reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration); + this.nsSummaryFlushToDBMaxThreshold = nsSummaryFlushToDBMaxThreshold; } @@ -92,14 +98,17 @@ public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { setKeyParentID(keyInfo); handlePutKeyEvent(keyInfo, nsSummaryMap); - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return false; + } } } } } catch (IOException ioEx) { LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ", ioEx); + nsSummaryMap.clear(); return false; } @@ -111,14 +120,23 @@ public boolean reprocessWithOBS(OMMetadataManager omMetadataManager) { return true; } - public boolean processWithOBS(OMUpdateEventBatch events) { + public Pair processWithOBS(OMUpdateEventBatch events, + int seekPos) { Iterator eventIterator = events.getIterator(); Map nsSummaryMap = new HashMap<>(); + int itrPos = 0; + while (eventIterator.hasNext() && itrPos < seekPos) { + eventIterator.next(); + itrPos++; + } + + int eventCounter = 0; while (eventIterator.hasNext()) { OMDBUpdateEvent omdbUpdateEvent = eventIterator.next(); OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction(); + eventCounter++; // We only process updates on OM's KeyTable String table = omdbUpdateEvent.getTable(); @@ -184,27 +202,27 @@ public boolean processWithOBS(OMUpdateEventBatch events) { default: LOG.debug("Skipping DB update event: {}", action); } - - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + if (nsSummaryMap.size() >= nsSummaryFlushToDBMaxThreshold) { + if (!flushAndCommitNSToDB(nsSummaryMap)) { + return new ImmutablePair<>(seekPos, false); + } + seekPos = eventCounter + 1; } } catch (IOException ioEx) { LOG.error("Unable to process Namespace Summary data in Recon DB. ", ioEx); - return false; - } - if (!checkAndCallFlushToDB(nsSummaryMap)) { - return false; + nsSummaryMap.clear(); + return new ImmutablePair<>(seekPos, false); } } // Flush and commit left-out entries at the end if (!flushAndCommitNSToDB(nsSummaryMap)) { - return false; + return new ImmutablePair<>(seekPos, false); } LOG.debug("Completed a process run of NSSummaryTaskWithOBS"); - return true; + return new ImmutablePair<>(seekPos, true); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 3dc91d0dfc94..2477e0935935 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 @@ -155,7 +155,8 @@ public Collection getTaskTables() { * @return Pair */ @Override - public Pair process(OMUpdateEventBatch events) { + public Pair> process(OMUpdateEventBatch events, + int seekPos) { Iterator eventIterator = events.getIterator(); // Initialize maps to store count and size information HashMap objectCountMap = initializeCountMap(); @@ -195,7 +196,8 @@ public Pair process(OMUpdateEventBatch events) { LOG.error( "Unexpected exception while processing the table {}, Action: {}", tableName, omdbUpdateEvent.getAction(), e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), + new ImmutablePair<>(0, false)); } } // Write the updated count and size information to the database @@ -209,7 +211,7 @@ public Pair process(OMUpdateEventBatch events) { writeDataToDB(replicatedSizeMap); } LOG.debug("Completed a 'process' run of OmTableInsightTask."); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } private void handlePutEvent(OMDBUpdateEvent event, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java index 2092d6a326c9..c843de7179b1 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java @@ -34,10 +34,13 @@ public interface ReconOmTask { /** * Process a set of OM events on tables that the task is listening on. + * * @param events Set of events to be processed by the task. - * @return Pair of task name -> task success. + * @param seekPosition position from where to start iterating events of events iterator. + * @return Pair of task name -> events iterator position, task success. */ - Pair process(OMUpdateEventBatch events); + Pair> process(OMUpdateEventBatch events, + int seekPosition); /** * Process a on tables that the task is listening on. diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java index 64405095855c..9c479cf0909c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java @@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicInteger; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -106,26 +107,28 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, try { if (!events.isEmpty()) { - Collection>> tasks = new ArrayList<>(); + Collection>>> tasks = + new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { ReconOmTask task = taskEntry.getValue(); // events passed to process method is no longer filtered - tasks.add(() -> task.process(events)); + tasks.add(() -> task.process(events, 0)); } - List>> results = + List>>> results = executorService.invokeAll(tasks); - List failedTasks = processTaskResults(results, events); + List> + failedTasks = processTaskResults(results, events); // Retry - List retryFailedTasks = new ArrayList<>(); + List> retryFailedTasks = new ArrayList<>(); if (!failedTasks.isEmpty()) { tasks.clear(); - for (String taskName : failedTasks) { - ReconOmTask task = reconOmTasks.get(taskName); + for (Pair taskPair : failedTasks) { + ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); // events passed to process method is no longer filtered - tasks.add(() -> task.process(events)); + tasks.add(() -> task.process(events, taskPair.getRight())); } results = executorService.invokeAll(tasks); retryFailedTasks = processTaskResults(results, events); @@ -133,14 +136,16 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, // Reprocess the failed tasks. if (!retryFailedTasks.isEmpty()) { - tasks.clear(); - for (String taskName : failedTasks) { - ReconOmTask task = reconOmTasks.get(taskName); - tasks.add(() -> task.reprocess(omMetadataManager)); + Collection>> reprocessTasks = + new ArrayList<>(); + for (Pair taskPair : failedTasks) { + ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); + reprocessTasks.add(() -> task.reprocess(omMetadataManager)); } - results = executorService.invokeAll(tasks); + List>> reprocessTaskResults = + executorService.invokeAll(reprocessTasks); List reprocessFailedTasks = - processTaskResults(results, events); + processTaskResultsForReprocess(reprocessTaskResults, events); ignoreFailedTasks(reprocessFailedTasks); } } @@ -233,6 +238,35 @@ public synchronized void stop() { } } + /** + * Wait on results of all tasks. + * + * @param results Set of Futures. + * @param events Events. + * @return List of failed task names with their seek position + * @throws ExecutionException execution Exception + * @throws InterruptedException Interrupted Exception + */ + private List> processTaskResults( + List>>> + results, + OMUpdateEventBatch events) + throws ExecutionException, InterruptedException { + List> failedTasks = new ArrayList<>(); + for (Future>> f : results) { + String taskName = f.get().getLeft(); + Pair right = f.get().getRight(); + if (!right.getRight()) { + LOG.info("Failed task : {}", taskName); + failedTasks.add(new ImmutablePair<>(taskName, right.getLeft())); + } else { + taskFailureCounter.get(taskName).set(0); + storeLastCompletedTransaction(taskName, events.getLastSequenceNumber()); + } + } + return failedTasks; + } + /** * Wait on results of all tasks. * @param results Set of Futures. @@ -241,15 +275,16 @@ public synchronized void stop() { * @throws ExecutionException execution Exception * @throws InterruptedException Interrupted Exception */ - private List processTaskResults(List>> - results, - OMUpdateEventBatch events) + private List processTaskResultsForReprocess( + List>> + results, + OMUpdateEventBatch events) throws ExecutionException, InterruptedException { List failedTasks = new ArrayList<>(); for (Future> f : results) { String taskName = f.get().getLeft(); if (!f.get().getRight()) { - LOG.info("Failed task : {}", taskName); + LOG.info("Failed reprocessing of task : {}", taskName); failedTasks.add(f.get().getLeft()); } else { taskFailureCounter.get(taskName).set(0); 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 da5484c9b898..df8146ac164b 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 @@ -479,7 +479,7 @@ public void testGetKeysForContainer() throws IOException { setUpFSOData(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, new OzoneConfiguration()); + reconOMMetadataManager, new OzoneConfiguration(), 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); // Reprocess the container key mapper to ensure the latest mapping is used reprocessContainerKeyMapper(); @@ -567,7 +567,7 @@ public void testGetKeysForContainerWithPrevKey() throws IOException { reprocessContainerKeyMapper(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, new OzoneConfiguration()); + reconOMMetadataManager, new OzoneConfiguration(), 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); response = containerEndpoint.getKeysForContainer(20L, -1, "/0/1/2/file7"); 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 a244e4ff2ce2..19856dfa942f 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 @@ -110,7 +110,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconOMMetadataManager, ozoneConfiguration, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java index 9cda6d6e4518..82db11e7e578 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 @@ -393,7 +393,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconOMMetadataManager, ozoneConfiguration, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java index dba245ce8b80..5fed74d21847 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java @@ -390,7 +390,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, - reconOMMetadataManager, conf); + reconOMMetadataManager, conf, 10); nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithOBSAndLegacy.java index 6a2f2c557db8..c58519f9ef31 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 @@ -386,11 +386,11 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithOBS nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS(reconNamespaceSummaryManager, - reconOMMetadataManager, conf); + reconOMMetadataManager, conf, 10); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, - reconOMMetadataManager, conf); + reconOMMetadataManager, conf, 10); nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); commonUtils = new CommonUtils(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOmDBInsightEndPoint.java index 61a9711876e7..613fe1e59370 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 @@ -323,13 +323,13 @@ public void setUp() throws Exception { setUpOmData(); nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconOMMetadataManager, ozoneConfiguration, 10); nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconOMMetadataManager, ozoneConfiguration, 10); nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconOMMetadataManager, ozoneConfiguration, 10); reconNamespaceSummaryManager.clearNSSummaryTable(); nSSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); nsSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestOpenKeysSearchEndpoint.java index f55d988cfe01..029b80ee5a01 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 @@ -122,7 +122,7 @@ public void setUp() throws Exception { populateOMDB(); NSSummaryTaskWithFSO nSSummaryTaskWithFso = new NSSummaryTaskWithFSO(reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconOMMetadataManager, ozoneConfiguration, 10); nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java index 0ad53cfe8406..319ff44764fa 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java @@ -54,11 +54,12 @@ public Collection getTaskTables() { } @Override - public Pair process(OMUpdateEventBatch events) { + public Pair> process( + OMUpdateEventBatch events, int seekPos) { if (++callCtr <= numFailuresAllowed) { - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } else { - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java index 67f962f18694..fce42a97827a 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java @@ -339,7 +339,7 @@ public void testKeyTableProcess() throws IOException { assertEquals(1, reconContainerMetadataManager.getKeyCountForContainer(3L)); // Process PUT & DELETE event. - containerKeyMapperTask.process(omUpdateEventBatch); + containerKeyMapperTask.process(omUpdateEventBatch, 0); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); @@ -430,7 +430,7 @@ public void testFileTableProcess() throws Exception { }); // Process PUT event for both the keys - containerKeyMapperTask.process(omUpdateEventBatch); + containerKeyMapperTask.process(omUpdateEventBatch, 0); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); @@ -463,7 +463,7 @@ public void testFileTableProcess() throws Exception { }); // Process DELETE event for key2 - containerKeyMapperTask.process(omUpdateEventBatch2); + containerKeyMapperTask.process(omUpdateEventBatch2, 0); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java index 3572f5813eff..e69baced2550 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java @@ -196,7 +196,7 @@ public void testProcess() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(Arrays.asList(event, event2)); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); // Verify 2 keys are in correct bins. assertEquals(2, fileCountBySizeDao.count()); @@ -251,7 +251,7 @@ public void testProcess() { omUpdateEventBatch = new OMUpdateEventBatch( Arrays.asList(updateEvent, putEvent, deleteEvent)); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); assertEquals(4, fileCountBySizeDao.count()); recordToFind.value3(1024L); @@ -389,7 +389,7 @@ public void testProcessAtScale() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); // Verify 2 keys are in correct bins. assertEquals(10000, fileCountBySizeDao.count()); @@ -465,7 +465,7 @@ public void testProcessAtScale() { } omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); assertEquals(10000, fileCountBySizeDao.count()); recordToFind = dslContext diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java index 485804240d52..a1e08e862b1f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java @@ -223,7 +223,7 @@ public class TestProcess { @BeforeEach public void setUp() throws IOException { nSSummaryTask.reprocess(reconOMMetadataManager); - nSSummaryTask.process(processEventBatch()); + nSSummaryTask.process(processEventBatch(), 0); nsSummaryForBucket1 = reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); 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 ba2e7497417e..f126c71b8900 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 @@ -51,6 +51,7 @@ import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -144,9 +145,11 @@ public static void setUp(@TempDir File tmpDir) throws Exception { populateOMDB(); + long nsSummaryFlushToDBMaxThreshold = ozoneConfiguration.getLong( + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, 10); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, - ozoneConfiguration); + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); } /** @@ -319,7 +322,7 @@ public class TestProcess { @BeforeEach public void setUp() throws IOException { nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); - nSSummaryTaskWithFso.processWithFSO(processEventBatch()); + nSSummaryTaskWithFso.processWithFSO(processEventBatch(), 0); } private OMUpdateEventBatch processEventBatch() throws IOException { diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java index 5ffd03cbb88a..b396bb862731 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java @@ -53,6 +53,8 @@ import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProvider; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -145,9 +147,11 @@ public static void setUp(@TempDir File tmpDir) throws Exception { populateOMDB(); + long nsSummaryFlushToDBMaxThreshold = omConfiguration.getLong( + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, 10); nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, - reconOMMetadataManager, omConfiguration); + reconOMMetadataManager, omConfiguration, nsSummaryFlushToDBMaxThreshold); } /** @@ -295,7 +299,7 @@ public class TestProcess { @BeforeEach public void setUp() throws IOException { nSSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); - nSSummaryTaskWithLegacy.processWithLegacy(processEventBatch()); + nSSummaryTaskWithLegacy.processWithLegacy(processEventBatch(), 0); nsSummaryForBucket1 = reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); @@ -694,6 +698,7 @@ private static void initializeNewOmMetadataManager( omDbDir.getAbsolutePath()); omConfiguration.set(OMConfigKeys .OZONE_OM_ENABLE_FILESYSTEM_PATHS, "true"); + omConfiguration.set(OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, "10"); omMetadataManager = new OmMetadataManagerImpl( omConfiguration, null); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java index db4803676390..48544c119cfc 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacyOBSLayout.java @@ -51,6 +51,7 @@ import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProviderWithFSO; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -118,6 +119,8 @@ public static void setUp(@TempDir File tmpDir) throws Exception { ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.setBoolean(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS, false); + ozoneConfiguration.setLong(OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, + 10); ReconTestInjector reconTestInjector = new ReconTestInjector.Builder(tmpDir) @@ -135,9 +138,12 @@ public static void setUp(@TempDir File tmpDir) throws Exception { populateOMDB(); + long nsSummaryFlushToDBMaxThreshold = ozoneConfiguration.getLong( + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, 10); nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconOMMetadataManager, ozoneConfiguration, + nsSummaryFlushToDBMaxThreshold); } /** @@ -243,7 +249,7 @@ public void setUp() throws IOException { // reinit Recon RocksDB's namespace CF. reconNamespaceSummaryManager.clearNSSummaryTable(); nSSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager); - nSSummaryTaskWithLegacy.processWithLegacy(processEventBatch()); + nSSummaryTaskWithLegacy.processWithLegacy(processEventBatch(), 0); nsSummaryForBucket1 = reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); 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 8f9d6b2990a5..ba06ae2b3f02 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 @@ -52,6 +52,8 @@ import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProviderWithFSO; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; @@ -130,9 +132,13 @@ public static void setUp(@TempDir File tmpDir) throws Exception { populateOMDB(); + long nsSummaryFlushToDBMaxThreshold = omConfiguration.getLong( + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); nSSummaryTaskWithOBS = new NSSummaryTaskWithOBS( reconNamespaceSummaryManager, - reconOMMetadataManager, omConfiguration); + reconOMMetadataManager, omConfiguration, + nsSummaryFlushToDBMaxThreshold); } /** @@ -238,7 +244,7 @@ public void setUp() throws IOException { // reinit Recon RocksDB's namespace CF. reconNamespaceSummaryManager.clearNSSummaryTable(); nSSummaryTaskWithOBS.reprocessWithOBS(reconOMMetadataManager); - nSSummaryTaskWithOBS.processWithOBS(processEventBatch()); + nSSummaryTaskWithOBS.processWithOBS(processEventBatch(), 0); nsSummaryForBucket1 = reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); @@ -464,6 +470,7 @@ private static void initializeNewOmMetadataManager( omDbDir.getAbsolutePath()); omConfiguration.set(OMConfigKeys .OZONE_OM_ENABLE_FILESYSTEM_PATHS, "true"); + omConfiguration.set(OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, "10"); omMetadataManager = new OmMetadataManagerImpl( omConfiguration, null); 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 56d8fe213152..5f7fec05bd16 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 @@ -66,6 +66,8 @@ import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeOpenKeyToOm; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeOpenFileToOm; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDeletedKeysToOm; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.DELETE; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.PUT; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.UPDATE; @@ -133,6 +135,8 @@ public TestOmTableInsightTask() { private void initializeInjector() throws IOException { ozoneConfiguration = new OzoneConfiguration(); + ozoneConfiguration.set(OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, + "10"); reconOMMetadataManager = getTestReconOmMetadataManager( initializeNewOmMetadataManager(Files.createDirectory( temporaryFolder.resolve("JunitOmDBDir")).toFile()), @@ -150,9 +154,12 @@ private void initializeInjector() throws IOException { omTableInsightTask = new OmTableInsightTask( globalStatsDao, getConfiguration(), reconOMMetadataManager); + long nsSummaryFlushToDBMaxThreshold = ozoneConfiguration.getLong( + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, - ozoneConfiguration); + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); dslContext = getDslContext(); } @@ -321,7 +328,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { DELETED_DIR_TABLE, PUT, null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents); - omTableInsightTask.process(putEventBatch); + omTableInsightTask.process(putEventBatch, 0); assertEquals(5, getCountForTable(DELETED_DIR_TABLE)); @@ -335,7 +342,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { getOmKeyInfo("vol1", "bucket1", DIR_ONE, 3L, false), DELETED_DIR_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents); - omTableInsightTask.process(deleteEventBatch); + omTableInsightTask.process(deleteEventBatch, 0); assertEquals(3, getCountForTable(DELETED_DIR_TABLE)); } @@ -471,7 +478,7 @@ public void testProcessForCount() { // Processing the initial batch of events OMUpdateEventBatch initialBatch = new OMUpdateEventBatch(initialEvents); - omTableInsightTask.process(initialBatch); + omTableInsightTask.process(initialBatch, 0); // Verifying the count in each table for (String tableName : omTableInsightTask.getTaskTables()) { @@ -500,7 +507,7 @@ public void testProcessForCount() { // Processing the additional events OMUpdateEventBatch additionalBatch = new OMUpdateEventBatch(additionalEvents); - omTableInsightTask.process(additionalBatch); + omTableInsightTask.process(additionalBatch, 0); // Verifying the final count in each table for (String tableName : omTableInsightTask.getTaskTables()) { if (tableName.equals(DELETED_TABLE)) { @@ -529,7 +536,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents); - omTableInsightTask.process(putEventBatch); + omTableInsightTask.process(putEventBatch, 0); // After 5 PUTs, size should be 5 * 1000 = 5000 for (String tableName : new ArrayList<>( @@ -547,7 +554,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { getOMUpdateEvent("item0", omKeyInfo, OPEN_FILE_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents); - omTableInsightTask.process(deleteEventBatch); + omTableInsightTask.process(deleteEventBatch, 0); // After deleting "item0", size should be 4 * 1000 = 4000 for (String tableName : new ArrayList<>( @@ -570,7 +577,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch updateEventBatch = new OMUpdateEventBatch(updateEvents); - omTableInsightTask.process(updateEventBatch); + omTableInsightTask.process(updateEventBatch, 0); // After updating "item1", size should be 4000 - 1000 + 2000 = 5000 // presentValue - oldValue + newValue = updatedValue @@ -607,7 +614,7 @@ public void testProcessForDeletedTable() { null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents); - omTableInsightTask.process(putEventBatch); + omTableInsightTask.process(putEventBatch, 0); // Each of the 5 RepeatedOmKeyInfo object has 5 OmKeyInfo obj, // so total deleted keys should be 5 * 5 = 25 assertEquals(25L, getCountForTable(DELETED_TABLE)); @@ -623,7 +630,7 @@ public void testProcessForDeletedTable() { getOMUpdateEvent("item0", repeatedOmKeyInfo, DELETED_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents); - omTableInsightTask.process(deleteEventBatch); + omTableInsightTask.process(deleteEventBatch, 0); // After deleting "item0" total deleted keys should be 20 assertEquals(20L, getCountForTable(DELETED_TABLE)); // After deleting "item0", size should be 4 * 1000 = 4000 diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java index b5e82a48a87d..48552b21caac 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertSame; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -74,8 +75,8 @@ public void testRegisterTask() { @Test public void testConsumeOMEvents() throws Exception { ReconOmTask reconOmTaskMock = getMockTask("MockTask"); - when(reconOmTaskMock.process(any(OMUpdateEventBatch.class))) - .thenReturn(new ImmutablePair<>("MockTask", true)); + when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyInt())) + .thenReturn(new ImmutablePair<>("MockTask", new ImmutablePair<>(0, true))); reconTaskController.registerTask(reconOmTaskMock); OMUpdateEventBatch omUpdateEventBatchMock = mock(OMUpdateEventBatch.class); when(omUpdateEventBatchMock.getLastSequenceNumber()).thenReturn(100L); @@ -87,7 +88,7 @@ public void testConsumeOMEvents() throws Exception { mock(OMMetadataManager.class)); verify(reconOmTaskMock, times(1)) - .process(any()); + .process(any(), anyInt()); long endTime = System.currentTimeMillis(); reconTaskStatusDao = getDao(ReconTaskStatusDao.class); From 5cd38b46b5e826ce758aee3be60066c7d301bef5 Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Mon, 20 Jan 2025 18:24:43 +0530 Subject: [PATCH 02/13] HDDS-12062. Code Refactoring. --- .../recon/tasks/ContainerKeyMapperTask.java | 19 +++--- .../ozone/recon/tasks/FileSizeCountTask.java | 13 ++-- .../ozone/recon/tasks/NSSummaryTask.java | 59 ++++++++++--------- .../ozone/recon/tasks/OmTableInsightTask.java | 15 ++--- .../hadoop/ozone/recon/tasks/ReconOmTask.java | 18 +----- .../recon/tasks/ReconTaskControllerImpl.java | 57 ++++++++++-------- .../hadoop/ozone/recon/api/TestEndpoints.java | 8 +-- .../ozone/recon/tasks/DummyReconDBTask.java | 6 +- .../recon/tasks/TestFileSizeCountTask.java | 16 ++--- .../recon/tasks/TestOmTableInsightTask.java | 40 ++++++------- .../tasks/TestReconTaskControllerImpl.java | 13 ++-- 11 files changed, 134 insertions(+), 130 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java index 3202f6aa8bbb..14baa5e918b8 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java @@ -85,7 +85,7 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager * (container, key) -> count to Recon Container DB. */ @Override - public Pair reprocess(OMMetadataManager omMetadataManager) { + public Pair> reprocess(OMMetadataManager omMetadataManager) { long omKeyCount = 0; // In-memory maps for fast look up and batch write @@ -121,7 +121,7 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { containerKeyCountMap); if (!checkAndCallFlushToDB(containerKeyMap)) { LOG.error("Unable to flush containerKey information to the DB"); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } omKeyCount++; } @@ -134,7 +134,7 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { containerKeyCountMap)) { LOG.error("Unable to flush Container Key Count and " + "remaining Container Key information to the DB"); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } LOG.debug("Completed 'reprocess' of ContainerKeyMapperTask."); @@ -145,9 +145,9 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { } catch (IOException ioEx) { LOG.error("Unable to populate Container Key data in Recon DB. ", ioEx); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } private boolean flushAndCommitContainerKeyInfoToDB( @@ -192,7 +192,8 @@ public Collection getTaskTables() { } @Override - public Pair process(OMUpdateEventBatch events) { + public Pair> process(OMUpdateEventBatch events, + int seekPosition) { Iterator eventIterator = events.getIterator(); int eventCount = 0; final Collection taskTables = getTaskTables(); @@ -249,18 +250,18 @@ public Pair process(OMUpdateEventBatch events) { } catch (IOException e) { LOG.error("Unexpected exception while updating key data : {} ", updatedKey, e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } } try { writeToTheDB(containerKeyMap, containerKeyCountMap, deletedKeyCountList); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } LOG.debug("{} successfully processed {} OM DB update event(s) in {} milliseconds.", getTaskName(), eventCount, (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } private void writeToTheDB(Map containerKeyMap, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java index 6906d4bbb828..b86f1be8454b 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java @@ -75,7 +75,7 @@ public FileSizeCountTask(FileCountBySizeDao fileCountBySizeDao, * @return Pair */ @Override - public Pair reprocess(OMMetadataManager omMetadataManager) { + public Pair> reprocess(OMMetadataManager omMetadataManager) { // Map to store the count of files based on file size Map fileSizeCountMap = new HashMap<>(); @@ -93,11 +93,11 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { reprocessBucketLayout(BucketLayout.LEGACY, omMetadataManager, fileSizeCountMap); if (!statusFSO && !statusOBS) { - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } writeCountsToDB(true, fileSizeCountMap); LOG.debug("Completed a 'reprocess' run of FileSizeCountTask."); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } private boolean reprocessBucketLayout(BucketLayout bucketLayout, @@ -144,7 +144,8 @@ public Collection getTaskTables() { * @return Pair */ @Override - public Pair process(OMUpdateEventBatch events) { + public Pair> process(OMUpdateEventBatch events, + int seekPosition) { Iterator eventIterator = events.getIterator(); Map fileSizeCountMap = new HashMap<>(); final Collection taskTables = getTaskTables(); @@ -191,7 +192,7 @@ public Pair process(OMUpdateEventBatch events) { } catch (Exception e) { LOG.error("Unexpected exception while processing key {}.", updatedKey, e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } } else { LOG.warn("Unexpected value type {} for key {}. Skipping processing.", @@ -201,7 +202,7 @@ public Pair process(OMUpdateEventBatch events) { writeCountsToDB(false, fileSizeCountMap); LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } /** diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index 5048050f9d48..c1a6abef5612 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 @@ -40,6 +40,9 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; +import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; + /** * Task to query data from OMDB and write into Recon RocksDB. * Reprocess() will take a snapshots on OMDB, and iterate the keyTable, @@ -70,7 +73,6 @@ public class NSSummaryTask implements ReconOmTask { private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO; private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy; private final NSSummaryTaskWithOBS nsSummaryTaskWithOBS; - private final OzoneConfiguration ozoneConfiguration; @Inject public NSSummaryTask(ReconNamespaceSummaryManager @@ -81,16 +83,19 @@ public NSSummaryTask(ReconNamespaceSummaryManager ozoneConfiguration) { 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); + this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO( - reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconNamespaceSummaryManager, reconOMMetadataManager, + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy( - reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconNamespaceSummaryManager, reconOMMetadataManager, + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); this.nsSummaryTaskWithOBS = new NSSummaryTaskWithOBS( - reconNamespaceSummaryManager, - reconOMMetadataManager, ozoneConfiguration); + reconNamespaceSummaryManager, reconOMMetadataManager, + ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); } @Override @@ -99,27 +104,28 @@ public String getTaskName() { } @Override - public Pair process(OMUpdateEventBatch events) { - long startTime = System.currentTimeMillis(); - boolean success = nsSummaryTaskWithFSO.processWithFSO(events); - if (!success) { + public Pair> process( + OMUpdateEventBatch events, int seekPosition) { + Pair itrPosStatusPair = + nsSummaryTaskWithFSO.processWithFSO(events, seekPosition); + if (!itrPosStatusPair.getRight()) { LOG.error("processWithFSO failed."); } - success = nsSummaryTaskWithLegacy.processWithLegacy(events); - if (!success) { + itrPosStatusPair = + nsSummaryTaskWithLegacy.processWithLegacy(events, seekPosition); + if (!itrPosStatusPair.getRight()) { LOG.error("processWithLegacy failed."); } - success = nsSummaryTaskWithOBS.processWithOBS(events); - if (!success) { + itrPosStatusPair = + nsSummaryTaskWithOBS.processWithOBS(events, seekPosition); + if (!itrPosStatusPair.getRight()) { LOG.error("processWithOBS failed."); } - LOG.debug("{} successfully processed in {} milliseconds", - getTaskName(), (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), success); + return new ImmutablePair<>(getTaskName(), itrPosStatusPair); } @Override - public Pair reprocess(OMMetadataManager omMetadataManager) { + public Pair> reprocess(OMMetadataManager omMetadataManager) { // Initialize a list of tasks to run in parallel Collection> tasks = new ArrayList<>(); @@ -131,7 +137,7 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { } catch (IOException ioEx) { LOG.error("Unable to clear NSSummary table in Recon DB. ", ioEx); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } tasks.add(() -> nsSummaryTaskWithFSO @@ -151,15 +157,12 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { results = executorService.invokeAll(tasks); for (int i = 0; i < results.size(); i++) { if (results.get(i).get().equals(false)) { - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } } - } catch (InterruptedException ex) { + } catch (InterruptedException | ExecutionException ex) { LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex); - return new ImmutablePair<>(getTaskName(), false); - } catch (ExecutionException ex2) { - LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex2); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } finally { executorService.shutdown(); @@ -172,7 +175,7 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { LOG.debug("Task execution time: {} milliseconds", durationInMillis); } - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 37a0e16e9346..8c7dee22d077 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 @@ -91,7 +91,7 @@ public OmTableInsightTask(GlobalStatsDao globalStatsDao, * @return Pair */ @Override - public Pair reprocess(OMMetadataManager omMetadataManager) { + public Pair> reprocess(OMMetadataManager omMetadataManager) { HashMap objectCountMap = initializeCountMap(); HashMap unReplicatedSizeMap = initializeSizeMap(false); HashMap replicatedSizeMap = initializeSizeMap(true); @@ -100,7 +100,7 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { Table table = omMetadataManager.getTable(tableName); if (table == null) { LOG.error("Table " + tableName + " not found in OM Metadata."); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } try (TableIterator> iterator @@ -120,7 +120,7 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { } } catch (IOException ioEx) { LOG.error("Unable to populate Table Count in Recon DB.", ioEx); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } } // Write the data to the DB @@ -135,7 +135,7 @@ public Pair reprocess(OMMetadataManager omMetadataManager) { } LOG.debug("Completed a 'reprocess' run of OmTableInsightTask."); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } @Override @@ -155,7 +155,8 @@ public Collection getTaskTables() { * @return Pair */ @Override - public Pair process(OMUpdateEventBatch events) { + public Pair> process(OMUpdateEventBatch events, + int seekPosition) { Iterator eventIterator = events.getIterator(); // Initialize maps to store count and size information HashMap objectCountMap = initializeCountMap(); @@ -196,7 +197,7 @@ public Pair process(OMUpdateEventBatch events) { LOG.error( "Unexpected exception while processing the table {}, Action: {}", tableName, omdbUpdateEvent.getAction(), e); - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } } // Write the updated count and size information to the database @@ -211,7 +212,7 @@ public Pair process(OMUpdateEventBatch events) { } LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } private void handlePutEvent(OMDBUpdateEvent event, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java index 218b82cfb6ac..bbd2c16b1131 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.recon.tasks; -import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -40,25 +39,14 @@ public interface ReconOmTask { * @param seekPosition position from where to start iterating events of events iterator. * @return Pair of task name -> events iterator position, task success. */ - default Pair> process( - OMUpdateEventBatch events, - int seekPosition) { - return new ImmutablePair<>(getTaskName(), Pair.of(seekPosition, true)); - } - - /** - * Process a set of OM events on tables that the task is listening on. - * - * @param events Set of events to be processed by the task. - * @return Pair of task name -> events iterator position, task success. - */ - Pair process(OMUpdateEventBatch events); + Pair> process(OMUpdateEventBatch events, + int seekPosition); /** * Process a on tables that the task is listening on. * @param omMetadataManager OM Metadata manager instance. * @return Pair of task name -> task success. */ - Pair reprocess(OMMetadataManager omMetadataManager); + Pair> reprocess(OMMetadataManager omMetadataManager); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java index 32b440badb7d..9b0fb4927cc6 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java @@ -37,6 +37,7 @@ import java.util.stream.Collectors; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -99,27 +100,27 @@ public void registerTask(ReconOmTask task) { @Override public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataManager omMetadataManager) { if (!events.isEmpty()) { - Collection>> tasks = new ArrayList<>(); - List failedTasks = new ArrayList<>(); + Collection>>> tasks = new ArrayList<>(); + List> failedTasks = new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { ReconOmTask task = taskEntry.getValue(); ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(task.getTaskName()); taskStatusUpdater.recordRunStart(); // events passed to process method is no longer filtered - tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.process(events))); + tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.process(events, 0))); } processTasks(tasks, events, failedTasks); // Retry processing failed tasks - List retryFailedTasks = new ArrayList<>(); + List> retryFailedTasks = new ArrayList<>(); if (!failedTasks.isEmpty()) { tasks.clear(); - for (String taskName : failedTasks) { - ReconOmTask task = reconOmTasks.get(taskName); + for (Pair taskPair : failedTasks) { + ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); // events passed to process method is no longer filtered tasks.add(new NamedCallableTask<>(task.getTaskName(), - () -> task.process(events))); + () -> task.process(events, taskPair.getRight()))); } processTasks(tasks, events, retryFailedTasks); } @@ -127,11 +128,11 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataMa // Reprocess the failed tasks. if (!retryFailedTasks.isEmpty()) { tasks.clear(); - for (String taskName : failedTasks) { - ReconOmTask task = reconOmTasks.get(taskName); + for (Pair taskPair : failedTasks) { + ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.reprocess(omMetadataManager))); } - List reprocessFailedTasks = new ArrayList<>(); + List> reprocessFailedTasks = new ArrayList<>(); processTasks(tasks, events, reprocessFailedTasks); ignoreFailedTasks(reprocessFailedTasks); } @@ -142,8 +143,9 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataMa * Ignore tasks that failed reprocess step more than threshold times. * @param failedTasks list of failed tasks. */ - private void ignoreFailedTasks(List failedTasks) { - for (String taskName : failedTasks) { + private void ignoreFailedTasks(List> failedTasks) { + for (Pair taskPair : failedTasks) { + String taskName = taskPair.getLeft(); LOG.info("Reprocess step failed for task {}.", taskName); if (taskFailureCounter.get(taskName).incrementAndGet() > TASK_FAILURE_THRESHOLD) { @@ -156,13 +158,15 @@ private void ignoreFailedTasks(List failedTasks) { @Override public synchronized void reInitializeTasks(ReconOMMetadataManager omMetadataManager) { - Collection>> tasks = new ArrayList<>(); + Collection>>> tasks = new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { ReconOmTask task = taskEntry.getValue(); - ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(task.getTaskName()); + ReconTaskStatusUpdater taskStatusUpdater = + taskStatusUpdaterManager.getTaskStatusUpdater(task.getTaskName()); taskStatusUpdater.recordRunStart(); - tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.reprocess(omMetadataManager))); + tasks.add(new NamedCallableTask<>(task.getTaskName(), + () -> task.reprocess(omMetadataManager))); } try { @@ -179,13 +183,15 @@ public synchronized void reInitializeTasks(ReconOMMetadataManager omMetadataMana } }, executorService).thenAccept(result -> { String taskName = result.getLeft(); - ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(taskName); - if (!result.getRight()) { + ReconTaskStatusUpdater taskStatusUpdater = + taskStatusUpdaterManager.getTaskStatusUpdater(taskName); + if (!result.getRight().getRight()) { LOG.error("Init failed for task {}.", taskName); taskStatusUpdater.setLastTaskRunStatus(-1); } else { taskStatusUpdater.setLastTaskRunStatus(0); - taskStatusUpdater.setLastUpdatedSeqNumber(omMetadataManager.getLastSequenceNumberFromDB()); + taskStatusUpdater.setLastUpdatedSeqNumber( + omMetadataManager.getLastSequenceNumberFromDB()); } taskStatusUpdater.recordRunCompletion(); }).exceptionally(ex -> { @@ -237,8 +243,9 @@ public synchronized void stop() { * @param events A batch of {@link OMUpdateEventBatch} events to fetch sequence number of last event in batch. * @param failedTasks Reference of the list to which we want to add the failed tasks for retry/reprocessing */ - private void processTasks(Collection>> tasks, - OMUpdateEventBatch events, List failedTasks) { + private void processTasks( + Collection>>> tasks, + OMUpdateEventBatch events, List> failedTasks) { List> futures = tasks.stream() .map(task -> CompletableFuture.supplyAsync(() -> { try { @@ -252,10 +259,11 @@ private void processTasks(Collection>> t } }, executorService).thenAccept(result -> { String taskName = result.getLeft(); - ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(taskName); - if (!result.getRight()) { + ReconTaskStatusUpdater taskStatusUpdater = + taskStatusUpdaterManager.getTaskStatusUpdater(taskName); + if (!result.getRight().getRight()) { LOG.error("Task {} failed", taskName); - failedTasks.add(result.getLeft()); + failedTasks.add(new ImmutablePair<>(result.getLeft(), result.getRight().getLeft())); taskStatusUpdater.setLastTaskRunStatus(-1); } else { taskFailureCounter.get(taskName).set(0); @@ -270,7 +278,8 @@ private void processTasks(Collection>> t String taskName = taskEx.getTaskName(); LOG.error("The above error occurred while trying to execute task: {}", taskName); - ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(taskName); + ReconTaskStatusUpdater taskStatusUpdater = + taskStatusUpdaterManager.getTaskStatusUpdater(taskName); taskStatusUpdater.setLastTaskRunStatus(-1); taskStatusUpdater.recordRunCompletion(); } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java index 5b6d22bf2671..b67997ceb5ad 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java @@ -788,9 +788,9 @@ public void testGetClusterState() throws Exception { }); // check volume, bucket and key count after running table count task - Pair result = + Pair> result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); response = clusterStateEndpoint.getClusterState(); clusterStateResponse = (ClusterStateResponse) response.getEntity(); assertEquals(2, clusterStateResponse.getVolumes()); @@ -867,9 +867,9 @@ public void testGetFileCounts() throws Exception { .thenReturn(omKeyInfo2) .thenReturn(omKeyInfo3); - Pair result = + Pair> result = fileSizeCountTask.reprocess(omMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); assertEquals(3, fileCountBySizeDao.count()); Response response = utilizationEndpoint.getFileCounts(null, null, 0); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java index 319ff44764fa..d65e02d36d78 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java @@ -64,11 +64,11 @@ public Pair> process( } @Override - public Pair reprocess(OMMetadataManager omMetadataManager) { + public Pair> reprocess(OMMetadataManager omMetadataManager) { if (++callCtr <= numFailuresAllowed) { - return new ImmutablePair<>(getTaskName(), false); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); } else { - return new ImmutablePair<>(getTaskName(), true); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); } } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java index e2e6f5c600c9..b60cc08ed324 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java @@ -137,11 +137,11 @@ public void testReprocess() throws IOException { fileCountBySizeDao.insert( new FileCountBySize("vol1", "bucket1", 1024L, 10L)); - Pair result = + Pair> result = fileSizeCountTask.reprocess(omMetadataManager); // Verify that the result of reprocess is true - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); // Verify that the number of entries in fileCountBySizeDao is 3 assertEquals(3, fileCountBySizeDao.count()); @@ -196,7 +196,7 @@ public void testProcess() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(Arrays.asList(event, event2), 0L); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); // Verify 2 keys are in correct bins. assertEquals(2, fileCountBySizeDao.count()); @@ -251,7 +251,7 @@ public void testProcess() { omUpdateEventBatch = new OMUpdateEventBatch( Arrays.asList(updateEvent, putEvent, deleteEvent), 0L); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); assertEquals(4, fileCountBySizeDao.count()); recordToFind.value3(1024L); @@ -324,9 +324,9 @@ public void testReprocessAtScale() throws IOException { when(mockKeyValueFso.getValue()) .thenAnswer(returnsElementsOf(omKeyInfoList)); - Pair result = + Pair> result = fileSizeCountTask.reprocess(omMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); // 2 volumes * 500 buckets * 42 bins = 42000 rows assertEquals(42000, fileCountBySizeDao.count()); @@ -389,7 +389,7 @@ public void testProcessAtScale() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList, 0L); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); // Verify 2 keys are in correct bins. assertEquals(10000, fileCountBySizeDao.count()); @@ -465,7 +465,7 @@ public void testProcessAtScale() { } omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList, 0L); - fileSizeCountTask.process(omUpdateEventBatch); + fileSizeCountTask.process(omUpdateEventBatch, 0); assertEquals(10000, fileCountBySizeDao.count()); recordToFind = dslContext 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 8bb4c1f724bd..210c439bce97 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 @@ -152,7 +152,7 @@ private void initializeInjector() throws IOException { globalStatsDao, getConfiguration(), reconOMMetadataManager); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, - ozoneConfiguration); + ozoneConfiguration, 10); dslContext = getDslContext(); } @@ -281,9 +281,9 @@ public void testReprocessForDeletedDirectory() throws Exception { // Generate NamespaceSummary for the OM DB nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); - Pair result = + Pair> result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); assertEquals(3, getCountForTable(DELETED_DIR_TABLE)); } @@ -321,7 +321,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { DELETED_DIR_TABLE, PUT, null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch); + omTableInsightTask.process(putEventBatch, 0); assertEquals(5, getCountForTable(DELETED_DIR_TABLE)); @@ -335,7 +335,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { getOmKeyInfo("vol1", "bucket1", DIR_ONE, 3L, false), DELETED_DIR_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch); + omTableInsightTask.process(deleteEventBatch, 0); assertEquals(3, getCountForTable(DELETED_DIR_TABLE)); } @@ -368,10 +368,10 @@ public void testReprocessForCount() throws Exception { when(mockIter.next()).thenReturn(mockKeyValue); } - Pair result = + Pair> result = omTableInsightTask.reprocess(omMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); assertEquals(5L, getCountForTable(KEY_TABLE)); assertEquals(5L, getCountForTable(VOLUME_TABLE)); assertEquals(5L, getCountForTable(BUCKET_TABLE)); @@ -389,9 +389,9 @@ public void testReprocessForOpenKeyTable() throws Exception { writeOpenKeyToOm(reconOMMetadataManager, "key1", "Bucket3", "Volume3", null, 3L); - Pair result = + Pair> result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); assertEquals(3L, getCountForTable(OPEN_KEY_TABLE)); // Test for both replicated and unreplicated size for OPEN_KEY_TABLE assertEquals(6L, getUnReplicatedSizeForTable(OPEN_KEY_TABLE)); @@ -408,9 +408,9 @@ public void testReprocessForOpenFileTable() throws Exception { writeOpenFileToOm(reconOMMetadataManager, "file3", "Bucket3", "Volume3", "file3", 3, 0, 3, 3, null, 3L); - Pair result = + Pair> result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); assertEquals(3L, getCountForTable(OPEN_FILE_TABLE)); // Test for both replicated and unreplicated size for OPEN_FILE_TABLE assertEquals(6L, getUnReplicatedSizeForTable(OPEN_FILE_TABLE)); @@ -432,9 +432,9 @@ public void testReprocessForDeletedTable() throws Exception { deletedKeysList3, "Bucket3", "Volume3"); - Pair result = + Pair> result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight()); + assertTrue(result.getRight().getRight()); assertEquals(6L, getCountForTable(DELETED_TABLE)); // Test for both replicated and unreplicated size for DELETED_TABLE assertEquals(600L, getUnReplicatedSizeForTable(DELETED_TABLE)); @@ -471,7 +471,7 @@ public void testProcessForCount() { // Processing the initial batch of events OMUpdateEventBatch initialBatch = new OMUpdateEventBatch(initialEvents, 0L); - omTableInsightTask.process(initialBatch); + omTableInsightTask.process(initialBatch, 0); // Verifying the count in each table for (String tableName : omTableInsightTask.getTaskTables()) { @@ -500,7 +500,7 @@ public void testProcessForCount() { // Processing the additional events OMUpdateEventBatch additionalBatch = new OMUpdateEventBatch(additionalEvents, 0L); - omTableInsightTask.process(additionalBatch); + omTableInsightTask.process(additionalBatch, 0); // Verifying the final count in each table for (String tableName : omTableInsightTask.getTaskTables()) { if (tableName.equals(DELETED_TABLE)) { @@ -529,7 +529,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch); + omTableInsightTask.process(putEventBatch, 0); // After 5 PUTs, size should be 5 * 1000 = 5000 for (String tableName : new ArrayList<>( @@ -547,7 +547,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { getOMUpdateEvent("item0", omKeyInfo, OPEN_FILE_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch); + omTableInsightTask.process(deleteEventBatch, 0); // After deleting "item0", size should be 4 * 1000 = 4000 for (String tableName : new ArrayList<>( @@ -570,7 +570,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch updateEventBatch = new OMUpdateEventBatch(updateEvents, 0L); - omTableInsightTask.process(updateEventBatch); + omTableInsightTask.process(updateEventBatch, 0); // After updating "item1", size should be 4000 - 1000 + 2000 = 5000 // presentValue - oldValue + newValue = updatedValue @@ -607,7 +607,7 @@ public void testProcessForDeletedTable() { null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch); + omTableInsightTask.process(putEventBatch, 0); // Each of the 5 RepeatedOmKeyInfo object has 5 OmKeyInfo obj, // so total deleted keys should be 5 * 5 = 25 assertEquals(25L, getCountForTable(DELETED_TABLE)); @@ -623,7 +623,7 @@ public void testProcessForDeletedTable() { getOMUpdateEvent("item0", repeatedOmKeyInfo, DELETED_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch); + omTableInsightTask.process(deleteEventBatch, 0); // After deleting "item0" total deleted keys should be 20 assertEquals(20L, getCountForTable(DELETED_TABLE)); // After deleting "item0", size should be 4 * 1000 = 4000 diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java index 477a5cace731..30f50c56da65 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java @@ -22,6 +22,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertSame; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -83,8 +84,8 @@ public void testRegisterTask() { @Test public void testConsumeOMEvents() throws Exception { ReconOmTask reconOmTaskMock = getMockTask("MockTask"); - when(reconOmTaskMock.process(any(OMUpdateEventBatch.class))) - .thenReturn(new ImmutablePair<>("MockTask", true)); + when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyInt())) + .thenReturn(new ImmutablePair<>("MockTask", new ImmutablePair<>(0, true))); reconTaskController.registerTask(reconOmTaskMock); OMUpdateEventBatch omUpdateEventBatchMock = mock(OMUpdateEventBatch.class); when(omUpdateEventBatchMock.getLastSequenceNumber()).thenReturn(100L); @@ -96,7 +97,7 @@ public void testConsumeOMEvents() throws Exception { mock(OMMetadataManager.class)); verify(reconOmTaskMock, times(1)) - .process(any()); + .process(any(), anyInt()); long endTime = System.currentTimeMillis(); ReconTaskStatus reconTaskStatus = reconTaskStatusDao.findById("MockTask"); @@ -113,7 +114,7 @@ public void testTaskRecordsFailureOnException() throws Exception { OMUpdateEventBatch omUpdateEventBatchMock = mock(OMUpdateEventBatch.class); // Throw exception when trying to run task - when(reconOmTaskMock.process(any(OMUpdateEventBatch.class))) + when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyInt())) .thenThrow(new RuntimeException("Mock Failure")); reconTaskController.registerTask(reconOmTaskMock); when(omUpdateEventBatchMock.getLastSequenceNumber()).thenReturn(100L); @@ -125,7 +126,7 @@ public void testTaskRecordsFailureOnException() throws Exception { mock(OMMetadataManager.class)); verify(reconOmTaskMock, times(1)) - .process(any()); + .process(any(), anyInt()); long endTime = System.currentTimeMillis(); ReconTaskStatus reconTaskStatus = reconTaskStatusDao.findById("MockTask"); @@ -211,7 +212,7 @@ public void testReInitializeTasks() throws Exception { ReconOmTask reconOmTaskMock = getMockTask("MockTask2"); when(reconOmTaskMock.reprocess(omMetadataManagerMock)) - .thenReturn(new ImmutablePair<>("MockTask2", true)); + .thenReturn(new ImmutablePair<>("MockTask2", new ImmutablePair<>(0, true))); when(omMetadataManagerMock.getLastSequenceNumberFromDB() ).thenReturn(100L); From dce0899fac92e19a11a164e1ad77f5885edf308e Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Tue, 21 Jan 2025 11:25:34 +0530 Subject: [PATCH 03/13] HDDS-12062. Added Junit tests. --- .../recon/tasks/TestNSSummaryTaskWithFSO.java | 110 +++++++++++++++++- 1 file changed, 108 insertions(+), 2 deletions(-) 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 057eb28b0a76..b213af5cc3c6 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.recon.tasks; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -25,6 +26,7 @@ import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.recon.ReconConstants; import org.apache.hadoop.ozone.recon.ReconTestInjector; @@ -32,19 +34,26 @@ import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider; import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mockito; import java.io.File; import java.io.IOException; +import java.lang.reflect.Field; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; +import java.util.Iterator; +import java.util.List; import java.util.Set; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProviderWithFSO; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.initializeNewOmMetadataManager; @@ -53,8 +62,13 @@ import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Test for NSSummaryTaskWithFSO. @@ -122,7 +136,7 @@ private TestNSSummaryTaskWithFSO() { public static void setUp(@TempDir File tmpDir) throws Exception { ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.setLong(OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, - 10); + 3); omMetadataManager = initializeNewOmMetadataManager(new File(tmpDir, "om")); OzoneManagerServiceProvider ozoneManagerServiceProvider = getMockOzoneManagerServiceProviderWithFSO(); @@ -146,7 +160,7 @@ public static void setUp(@TempDir File tmpDir) throws Exception { populateOMDB(); long nsSummaryFlushToDBMaxThreshold = ozoneConfiguration.getLong( - OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, 10); + OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD, 3); nSSummaryTaskWithFso = new NSSummaryTaskWithFSO( reconNamespaceSummaryManager, reconOMMetadataManager, ozoneConfiguration, nsSummaryFlushToDBMaxThreshold); @@ -517,6 +531,98 @@ public void testParentIdAfterProcessEventBatch() throws IOException { "DIR_FIVE's parent ID should match BUCKET_TWO_OBJECT_ID."); } + @Test + void testProcessWithFSOFlushAfterThresholdAndSuccess() throws IOException { + // Call the method under test + Pair result = nSSummaryTaskWithFso.processWithFSO(processEventBatch(), 0); + + // Assertions + Assertions.assertNotNull(result, "Result should not be null"); + // Why seekPos should be 7 ? because we have threshold value for flush is set as 3, + // and we have total 7 events, so nsSummaryMap will be flushed in 2 batches and + // during second batch flush, eventCounter will be 6, then last event7 alone will + // be flushed out of loop as remaining event. At every batch flush based on threshold, + // seekPos is set as equal to eventCounter + 1, so seekPos will be 7. + Assertions.assertEquals(7, result.getLeft(), "seekPos should be 7"); + Assertions.assertTrue(result.getRight(), "The processing should fail due to flush failure"); + } + + @Test + void testProcessWithFSOFlushAfterThresholdAndFailureOfLastElement() + throws IOException, NoSuchFieldException, IllegalAccessException { + // Assume the NamespaceSummaryTaskWithFSO object is already created + NSSummaryTaskWithFSO task = mock(NSSummaryTaskWithFSO.class); + + // Set the value of nsSummaryFlushToDBMaxThreshold to 3 using reflection + Field thresholdField = NSSummaryTaskWithFSO.class.getDeclaredField("nsSummaryFlushToDBMaxThreshold"); + thresholdField.setAccessible(true); + thresholdField.set(task, 3); + + ReconNamespaceSummaryManager mockReconNamespaceSummaryManager = Mockito.mock(ReconNamespaceSummaryManager.class); + Field managerField = NSSummaryTaskDbEventHandler.class.getDeclaredField("reconNamespaceSummaryManager"); + managerField.setAccessible(true); + managerField.set(task, mockReconNamespaceSummaryManager); + + // Mock the OMUpdateEventBatch and its iterator + OMUpdateEventBatch events = Mockito.mock(OMUpdateEventBatch.class); + Iterator mockIterator = Mockito.mock(Iterator.class); + + Mockito.when(events.getIterator()).thenReturn(mockIterator); + + // Mock OMDBUpdateEvent objects and their behavior + OMDBUpdateEvent event1 = Mockito.mock(OMDBUpdateEvent.class); + OMDBUpdateEvent event2 = Mockito.mock(OMDBUpdateEvent.class); + OMDBUpdateEvent event3 = Mockito.mock(OMDBUpdateEvent.class); + OMDBUpdateEvent event4 = Mockito.mock(OMDBUpdateEvent.class); + + // Mock getAction() for each event + Mockito.when(event1.getAction()).thenReturn(OMDBUpdateEvent.OMDBUpdateAction.PUT); + Mockito.when(event2.getAction()).thenReturn(OMDBUpdateEvent.OMDBUpdateAction.PUT); + Mockito.when(event3.getAction()).thenReturn(OMDBUpdateEvent.OMDBUpdateAction.PUT); + Mockito.when(event4.getAction()).thenReturn(OMDBUpdateEvent.OMDBUpdateAction.PUT); + + OmKeyInfo keyInfo1 = new OmKeyInfo.Builder().setParentObjectID(1).setObjectID(2).setKeyName("key1") + .setBucketName("bucket1") + .setDataSize(1024).setVolumeName("volume1").build(); + OmKeyInfo keyInfo2 = new OmKeyInfo.Builder().setParentObjectID(1).setObjectID(3).setKeyName("key2") + .setBucketName("bucket1") + .setDataSize(1024).setVolumeName("volume1").build(); + OmKeyInfo keyInfo3 = new OmKeyInfo.Builder().setParentObjectID(1).setObjectID(3).setKeyName("key2") + .setBucketName("bucket1") + .setDataSize(1024).setVolumeName("volume1").build(); + OmKeyInfo keyInfo4 = new OmKeyInfo.Builder().setParentObjectID(1).setObjectID(3).setKeyName("key2") + .setBucketName("bucket1") + .setDataSize(1024).setVolumeName("volume1").build(); + Mockito.when(event1.getValue()).thenReturn(keyInfo1); + Mockito.when(event2.getValue()).thenReturn(keyInfo2); + Mockito.when(event3.getValue()).thenReturn(keyInfo3); + Mockito.when(event4.getValue()).thenReturn(keyInfo4); + + // Mock getTable() to return valid table name + Mockito.when(event1.getTable()).thenReturn(FILE_TABLE); + Mockito.when(event2.getTable()).thenReturn(FILE_TABLE); + Mockito.when(event3.getTable()).thenReturn(FILE_TABLE); + Mockito.when(event4.getTable()).thenReturn(FILE_TABLE); + + // Mock iterator to return the events + Mockito.when(mockIterator.hasNext()).thenReturn(true, true, true, true, false); + Mockito.when(mockIterator.next()).thenReturn(event1, event2, event3, event4); + + // Mock the flushAndCommitNSToDB method to fail on the last flush + NSSummaryTaskWithFSO taskSpy = Mockito.spy(task); + Mockito.doReturn(true).doReturn(true).doReturn(false).when(taskSpy).flushAndCommitNSToDB(Mockito.anyMap()); + + // Call the method under test + Pair result = taskSpy.processWithFSO(events, 0); + + // Assertions + Assertions.assertNotNull(result, "Result should not be null"); + Assertions.assertEquals(0, result.getLeft(), "seekPos should be 4"); + + // Verify interactions + Mockito.verify(mockIterator, Mockito.times(3)).next(); + Mockito.verify(taskSpy, Mockito.times(1)).flushAndCommitNSToDB(Mockito.anyMap()); + } } /** From df071258e649cf84aa5a2bb43b3308786986029b Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Tue, 21 Jan 2025 11:31:36 +0530 Subject: [PATCH 04/13] HDDS-12062. Fixed checkstyle issues. --- .../hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java | 1 - .../ozone/recon/tasks/TestNSSummaryTaskWithFSO.java | 8 -------- .../ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java | 1 - 3 files changed, 10 deletions(-) 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 2d49578bd08c..185c50ebd64f 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 @@ -37,7 +37,6 @@ import java.util.Collection; import java.util.HashMap; import java.util.Iterator; -import java.util.List; import java.util.Map; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE; 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 b213af5cc3c6..38e75323bbaa 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 @@ -26,7 +26,6 @@ import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; -import org.apache.hadoop.ozone.om.helpers.OmKeyArgs; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.recon.ReconConstants; import org.apache.hadoop.ozone.recon.ReconTestInjector; @@ -45,11 +44,9 @@ import java.io.File; import java.io.IOException; import java.lang.reflect.Field; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Set; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; @@ -60,15 +57,10 @@ import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; -import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.mockito.ArgumentMatchers.argThat; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Test for NSSummaryTaskWithFSO. diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java index d4bdb939a3a8..432004dad1cf 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java @@ -54,7 +54,6 @@ import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProvider; import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD; -import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_NSSUMMARY_FLUSH_TO_DB_MAX_THRESHOLD_DEFAULT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertNotNull; From 4fff1d4cd276081dc1331e119583fe3851c3a944 Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Tue, 21 Jan 2025 12:39:44 +0530 Subject: [PATCH 05/13] HDDS-12062. Fixed findbugs issues. --- .../hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 060dce3bea28..27fcf4831b94 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 @@ -73,10 +73,10 @@ public ReconOMMetadataManager getReconOMMetadataManager() { protected void writeNSSummariesToDB(Map nsSummaryMap) throws IOException { try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) { - for (Long key : nsSummaryMap.keySet()) { + for (Map.Entry entry : nsSummaryMap.entrySet()) { try { reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, - key, nsSummaryMap.get(key)); + entry.getKey(), entry.getValue()); } catch (IOException e) { LOG.error("Unable to write Namespace Summary data in Recon DB.", e); From 46969703cb03d5ba9e0101772998667126b6817d Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Tue, 21 Jan 2025 15:02:09 +0530 Subject: [PATCH 06/13] HDDS-12062. Fixed testcase issues. --- .../hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 38e75323bbaa..9eab91a232b9 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 @@ -325,10 +325,12 @@ public class TestProcess { private OMDBUpdateEvent keyEvent6; private OMDBUpdateEvent keyEvent7; + private Pair result; + @BeforeEach public void setUp() throws IOException { nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); - nSSummaryTaskWithFso.processWithFSO(processEventBatch(), 0); + result = nSSummaryTaskWithFso.processWithFSO(processEventBatch(), 0); } private OMUpdateEventBatch processEventBatch() throws IOException { @@ -526,7 +528,6 @@ public void testParentIdAfterProcessEventBatch() throws IOException { @Test void testProcessWithFSOFlushAfterThresholdAndSuccess() throws IOException { // Call the method under test - Pair result = nSSummaryTaskWithFso.processWithFSO(processEventBatch(), 0); // Assertions Assertions.assertNotNull(result, "Result should not be null"); @@ -541,7 +542,7 @@ void testProcessWithFSOFlushAfterThresholdAndSuccess() throws IOException { @Test void testProcessWithFSOFlushAfterThresholdAndFailureOfLastElement() - throws IOException, NoSuchFieldException, IllegalAccessException { + throws NoSuchFieldException, IllegalAccessException { // Assume the NamespaceSummaryTaskWithFSO object is already created NSSummaryTaskWithFSO task = mock(NSSummaryTaskWithFSO.class); From 034cd19887424ac2a511bfe174f85905c49418c0 Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Tue, 21 Jan 2025 21:36:32 +0530 Subject: [PATCH 07/13] HDDS-12062. Fixed checkstyle issues. --- .../hadoop/ozone/recon/tasks/TestNSSummaryTaskWithFSO.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 9eab91a232b9..f1396adc69bd 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 @@ -606,11 +606,11 @@ void testProcessWithFSOFlushAfterThresholdAndFailureOfLastElement() Mockito.doReturn(true).doReturn(true).doReturn(false).when(taskSpy).flushAndCommitNSToDB(Mockito.anyMap()); // Call the method under test - Pair result = taskSpy.processWithFSO(events, 0); + Pair result1 = taskSpy.processWithFSO(events, 0); // Assertions - Assertions.assertNotNull(result, "Result should not be null"); - Assertions.assertEquals(0, result.getLeft(), "seekPos should be 4"); + Assertions.assertNotNull(result1, "Result should not be null"); + Assertions.assertEquals(0, result1.getLeft(), "seekPos should be 4"); // Verify interactions Mockito.verify(mockIterator, Mockito.times(3)).next(); From 663f207da06eba98a37f669fcce746482b6ece6f Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Sat, 8 Feb 2025 19:58:37 +0530 Subject: [PATCH 08/13] HDDS-12062. Fixed review comments. --- .../recon/tasks/ContainerKeyMapperTask.java | 20 +++--- .../ozone/recon/tasks/FileSizeCountTask.java | 17 ++--- .../ozone/recon/tasks/NSSummaryTask.java | 71 ++++++++++++++----- .../tasks/NSSummaryTaskDbEventHandler.java | 10 --- .../ozone/recon/tasks/OmTableInsightTask.java | 21 +++--- .../hadoop/ozone/recon/tasks/ReconOmTask.java | 20 +++--- .../recon/tasks/ReconTaskControllerImpl.java | 29 ++++---- .../hadoop/ozone/recon/api/TestEndpoints.java | 4 +- .../ozone/recon/tasks/DummyReconDBTask.java | 16 +++-- .../tasks/TestContainerKeyMapperTask.java | 7 +- .../recon/tasks/TestFileSizeCountTask.java | 14 ++-- .../ozone/recon/tasks/TestNSSummaryTask.java | 3 +- .../recon/tasks/TestOmTableInsightTask.java | 30 ++++---- .../tasks/TestReconTaskControllerImpl.java | 15 ++-- 14 files changed, 161 insertions(+), 116 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java index 14baa5e918b8..cbfcdfdad232 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java @@ -85,7 +85,7 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager * (container, key) -> count to Recon Container DB. */ @Override - public Pair> reprocess(OMMetadataManager omMetadataManager) { + public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { long omKeyCount = 0; // In-memory maps for fast look up and batch write @@ -121,7 +121,7 @@ public Pair> reprocess(OMMetadataManager omMetada containerKeyCountMap); if (!checkAndCallFlushToDB(containerKeyMap)) { LOG.error("Unable to flush containerKey information to the DB"); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } omKeyCount++; } @@ -134,7 +134,7 @@ public Pair> reprocess(OMMetadataManager omMetada containerKeyCountMap)) { LOG.error("Unable to flush Container Key Count and " + "remaining Container Key information to the DB"); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } LOG.debug("Completed 'reprocess' of ContainerKeyMapperTask."); @@ -145,9 +145,9 @@ public Pair> reprocess(OMMetadataManager omMetada } catch (IOException ioEx) { LOG.error("Unable to populate Container Key data in Recon DB. ", ioEx); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } private boolean flushAndCommitContainerKeyInfoToDB( @@ -192,8 +192,8 @@ public Collection getTaskTables() { } @Override - public Pair> process(OMUpdateEventBatch events, - int seekPosition) { + public Pair, Boolean>> process(OMUpdateEventBatch events, + Map subTaskSeekPosMap) { Iterator eventIterator = events.getIterator(); int eventCount = 0; final Collection taskTables = getTaskTables(); @@ -250,18 +250,18 @@ public Pair> process(OMUpdateEventBatch events, } catch (IOException e) { LOG.error("Unexpected exception while updating key data : {} ", updatedKey, e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } } try { writeToTheDB(containerKeyMap, containerKeyCountMap, deletedKeyCountList); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } LOG.debug("{} successfully processed {} OM DB update event(s) in {} milliseconds.", getTaskName(), eventCount, (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } private void writeToTheDB(Map containerKeyMap, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java index b86f1be8454b..9d0c33bd94f3 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java @@ -75,7 +75,7 @@ public FileSizeCountTask(FileCountBySizeDao fileCountBySizeDao, * @return Pair */ @Override - public Pair> reprocess(OMMetadataManager omMetadataManager) { + public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { // Map to store the count of files based on file size Map fileSizeCountMap = new HashMap<>(); @@ -93,11 +93,11 @@ public Pair> reprocess(OMMetadataManager omMetada reprocessBucketLayout(BucketLayout.LEGACY, omMetadataManager, fileSizeCountMap); if (!statusFSO && !statusOBS) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } writeCountsToDB(true, fileSizeCountMap); LOG.debug("Completed a 'reprocess' run of FileSizeCountTask."); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } private boolean reprocessBucketLayout(BucketLayout bucketLayout, @@ -140,12 +140,13 @@ public Collection getTaskTables() { * Read the Keys from update events and update the count of files * pertaining to a certain upper bound. * - * @param events Update events - PUT/DELETE. + * @param events Update events - PUT/DELETE. + * @param subTaskSeekPosMap * @return Pair */ @Override - public Pair> process(OMUpdateEventBatch events, - int seekPosition) { + public Pair, Boolean>> process(OMUpdateEventBatch events, + Map subTaskSeekPosMap) { Iterator eventIterator = events.getIterator(); Map fileSizeCountMap = new HashMap<>(); final Collection taskTables = getTaskTables(); @@ -192,7 +193,7 @@ public Pair> process(OMUpdateEventBatch events, } catch (Exception e) { LOG.error("Unexpected exception while processing key {}.", updatedKey, e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } } else { LOG.warn("Unexpected value type {} for key {}. Skipping processing.", @@ -202,7 +203,7 @@ public Pair> process(OMUpdateEventBatch events, writeCountsToDB(false, fileSizeCountMap); LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } /** diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index c1a6abef5612..6b16b902b1a8 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 @@ -31,7 +31,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ExecutionException; @@ -103,29 +105,64 @@ public String getTaskName() { return "NSSummaryTask"; } + /** + * Bucket Type Enum which mimic subtasks for their data processing. + */ + public enum BucketType { + FSO("File System Optimized Bucket"), + OBS("Object Store Bucket"), + LEGACY("Legacy Bucket"); + + private final String description; + + BucketType(String description) { + this.description = description; + } + + public String getDescription() { + return description; + } + } + @Override - public Pair> process( - OMUpdateEventBatch events, int seekPosition) { - Pair itrPosStatusPair = - nsSummaryTaskWithFSO.processWithFSO(events, seekPosition); - if (!itrPosStatusPair.getRight()) { + public Pair, Boolean>> process( + OMUpdateEventBatch events, Map subTaskSeekPosMap) { + boolean anyFailure = false; // Track if any bucket fails + Map updatedSeekPositions = new HashMap<>(); + + // Process FSO bucket + Integer bucketSeek = subTaskSeekPosMap.getOrDefault(BucketType.FSO.name(), 0); + Pair bucketResult = nsSummaryTaskWithFSO.processWithFSO(events, bucketSeek); + updatedSeekPositions.put(BucketType.FSO.name(), bucketResult.getLeft()); + if (!bucketResult.getRight()) { LOG.error("processWithFSO failed."); + anyFailure = true; } - itrPosStatusPair = - nsSummaryTaskWithLegacy.processWithLegacy(events, seekPosition); - if (!itrPosStatusPair.getRight()) { + + // Process Legacy bucket + bucketSeek = subTaskSeekPosMap.getOrDefault(BucketType.LEGACY.name(), 0); + bucketResult = nsSummaryTaskWithLegacy.processWithLegacy(events, bucketSeek); + updatedSeekPositions.put(BucketType.LEGACY.name(), bucketResult.getLeft()); + if (!bucketResult.getRight()) { LOG.error("processWithLegacy failed."); + anyFailure = true; } - itrPosStatusPair = - nsSummaryTaskWithOBS.processWithOBS(events, seekPosition); - if (!itrPosStatusPair.getRight()) { + + // Process OBS bucket + bucketSeek = subTaskSeekPosMap.getOrDefault(BucketType.OBS.name(), 0); + bucketResult = nsSummaryTaskWithOBS.processWithOBS(events, bucketSeek); + updatedSeekPositions.put(BucketType.OBS.name(), bucketResult.getLeft()); + if (!bucketResult.getRight()) { LOG.error("processWithOBS failed."); + anyFailure = true; } - return new ImmutablePair<>(getTaskName(), itrPosStatusPair); + + // Return task failure if any bucket failed, while keeping each bucket's latest seek position + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(updatedSeekPositions, !anyFailure)); } @Override - public Pair> reprocess(OMMetadataManager omMetadataManager) { + public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { // Initialize a list of tasks to run in parallel Collection> tasks = new ArrayList<>(); @@ -137,7 +174,7 @@ public Pair> reprocess(OMMetadataManager omMetada } catch (IOException ioEx) { LOG.error("Unable to clear NSSummary table in Recon DB. ", ioEx); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } tasks.add(() -> nsSummaryTaskWithFSO @@ -157,12 +194,12 @@ public Pair> reprocess(OMMetadataManager omMetada results = executorService.invokeAll(tasks); for (int i = 0; i < results.size(); i++) { if (results.get(i).get().equals(false)) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } } } catch (InterruptedException | ExecutionException ex) { LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } finally { executorService.shutdown(); @@ -175,7 +212,7 @@ public Pair> reprocess(OMMetadataManager omMetada LOG.debug("Task execution time: {} milliseconds", durationInMillis); } - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } } 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 27fcf4831b94..4353b401b814 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 @@ -219,14 +219,4 @@ protected boolean flushAndCommitNSToDB(Map nsSummaryMap) { } return true; } - -/* protected boolean checkAndCallFlushToDB( - Map nsSummaryMap) { - // if map contains more than entries, flush to DB and clear the map - if (null != nsSummaryMap && nsSummaryMap.size() >= - nsSummaryFlushToDBMaxThreshold) { - return flushAndCommitNSToDB(nsSummaryMap); - } - return true; - }*/ } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 8c7dee22d077..1e55b74f5d1f 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 @@ -81,7 +81,7 @@ public OmTableInsightTask(GlobalStatsDao globalStatsDao, /** * Iterates the rows of each table in the OM snapshot DB and calculates the * counts and sizes for table data. - * + *

* For tables that require data size calculation * (as returned by getTablesToCalculateSize), both the number of * records (count) and total data size of the records are calculated. @@ -91,7 +91,7 @@ public OmTableInsightTask(GlobalStatsDao globalStatsDao, * @return Pair */ @Override - public Pair> reprocess(OMMetadataManager omMetadataManager) { + public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { HashMap objectCountMap = initializeCountMap(); HashMap unReplicatedSizeMap = initializeSizeMap(false); HashMap replicatedSizeMap = initializeSizeMap(true); @@ -100,7 +100,7 @@ public Pair> reprocess(OMMetadataManager omMetada Table table = omMetadataManager.getTable(tableName); if (table == null) { LOG.error("Table " + tableName + " not found in OM Metadata."); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } try (TableIterator> iterator @@ -120,7 +120,7 @@ public Pair> reprocess(OMMetadataManager omMetada } } catch (IOException ioEx) { LOG.error("Unable to populate Table Count in Recon DB.", ioEx); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } } // Write the data to the DB @@ -135,7 +135,7 @@ public Pair> reprocess(OMMetadataManager omMetada } LOG.debug("Completed a 'reprocess' run of OmTableInsightTask."); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } @Override @@ -151,12 +151,13 @@ public Collection getTaskTables() { * Read the update events and update the count and sizes of respective object * (volume, bucket, key etc.) based on the action (put or delete). * - * @param events Update events - PUT, DELETE and UPDATE. + * @param events Update events - PUT, DELETE and UPDATE. + * @param subTaskSeekPosMap * @return Pair */ @Override - public Pair> process(OMUpdateEventBatch events, - int seekPosition) { + public Pair, Boolean>> process(OMUpdateEventBatch events, + Map subTaskSeekPosMap) { Iterator eventIterator = events.getIterator(); // Initialize maps to store count and size information HashMap objectCountMap = initializeCountMap(); @@ -197,7 +198,7 @@ public Pair> process(OMUpdateEventBatch events, LOG.error( "Unexpected exception while processing the table {}, Action: {}", tableName, omdbUpdateEvent.getAction(), e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } } // Write the updated count and size information to the database @@ -212,7 +213,7 @@ public Pair> process(OMUpdateEventBatch events, } LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } private void handlePutEvent(OMDBUpdateEvent event, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java index bbd2c16b1131..de08c2f05cf1 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java @@ -21,6 +21,8 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.ozone.om.OMMetadataManager; +import java.util.Map; + /** * Interface used to denote a Recon task that needs to act on OM DB events. */ @@ -33,20 +35,22 @@ public interface ReconOmTask { String getTaskName(); /** - * Process a set of OM events on tables that the task is listening on. + * Processes a set of OM events on tables that the task is listening to. * - * @param events Set of events to be processed by the task. - * @param seekPosition position from where to start iterating events of events iterator. - * @return Pair of task name -> events iterator position, task success. + * @param events Set of events to be processed by the task. + * @param subTaskSeekPosMap Position from where to start iterating events of the event iterator for a given sub-task. + * @return A pair containing the task name mapped to a map of + * {@code sub-task name, its events iterator position} and a task success flag. */ - Pair> process(OMUpdateEventBatch events, - int seekPosition); + Pair, Boolean>> process(OMUpdateEventBatch events, + Map subTaskSeekPosMap); /** * Process a on tables that the task is listening on. + * * @param omMetadataManager OM Metadata manager instance. - * @return Pair of task name -> task success. + * @return Pair of task name -> map of , task success. */ - Pair> reprocess(OMMetadataManager omMetadataManager); + Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager); } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java index 9b0fb4927cc6..4868b51df81b 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java @@ -100,23 +100,23 @@ public void registerTask(ReconOmTask task) { @Override public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataManager omMetadataManager) { if (!events.isEmpty()) { - Collection>>> tasks = new ArrayList<>(); - List> failedTasks = new ArrayList<>(); + Collection, Boolean>>>> tasks = new ArrayList<>(); + List>> failedTasks = new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { ReconOmTask task = taskEntry.getValue(); ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(task.getTaskName()); taskStatusUpdater.recordRunStart(); // events passed to process method is no longer filtered - tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.process(events, 0))); + tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.process(events, new HashMap<>()))); } processTasks(tasks, events, failedTasks); // Retry processing failed tasks - List> retryFailedTasks = new ArrayList<>(); + List>> retryFailedTasks = new ArrayList<>(); if (!failedTasks.isEmpty()) { tasks.clear(); - for (Pair taskPair : failedTasks) { + for (Pair> taskPair : failedTasks) { ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); // events passed to process method is no longer filtered tasks.add(new NamedCallableTask<>(task.getTaskName(), @@ -128,12 +128,15 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataMa // Reprocess the failed tasks. if (!retryFailedTasks.isEmpty()) { tasks.clear(); - for (Pair taskPair : failedTasks) { + for (Pair> taskPair : failedTasks) { ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.reprocess(omMetadataManager))); } - List> reprocessFailedTasks = new ArrayList<>(); + List>> reprocessFailedTasks = new ArrayList<>(); processTasks(tasks, events, reprocessFailedTasks); + // Here the assumption is that even if full re-process of task also fails, + // then there is something wrong in recon rocks DB got from OM and needs to be + // investigated. ignoreFailedTasks(reprocessFailedTasks); } } @@ -143,8 +146,8 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataMa * Ignore tasks that failed reprocess step more than threshold times. * @param failedTasks list of failed tasks. */ - private void ignoreFailedTasks(List> failedTasks) { - for (Pair taskPair : failedTasks) { + private void ignoreFailedTasks(List>> failedTasks) { + for (Pair> taskPair : failedTasks) { String taskName = taskPair.getLeft(); LOG.info("Reprocess step failed for task {}.", taskName); if (taskFailureCounter.get(taskName).incrementAndGet() > @@ -158,7 +161,7 @@ private void ignoreFailedTasks(List> failedTasks) { @Override public synchronized void reInitializeTasks(ReconOMMetadataManager omMetadataManager) { - Collection>>> tasks = new ArrayList<>(); + Collection, Boolean>>>> tasks = new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { ReconOmTask task = taskEntry.getValue(); @@ -244,8 +247,8 @@ public synchronized void stop() { * @param failedTasks Reference of the list to which we want to add the failed tasks for retry/reprocessing */ private void processTasks( - Collection>>> tasks, - OMUpdateEventBatch events, List> failedTasks) { + Collection, Boolean>>>> tasks, + OMUpdateEventBatch events, List>> failedTasks) { List> futures = tasks.stream() .map(task -> CompletableFuture.supplyAsync(() -> { try { @@ -263,7 +266,7 @@ private void processTasks( taskStatusUpdaterManager.getTaskStatusUpdater(taskName); if (!result.getRight().getRight()) { LOG.error("Task {} failed", taskName); - failedTasks.add(new ImmutablePair<>(result.getLeft(), result.getRight().getLeft())); + failedTasks.add(new ImmutablePair<>(taskName, result.getRight().getLeft())); taskStatusUpdater.setLastTaskRunStatus(-1); } else { taskFailureCounter.get(taskName).set(0); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java index b67997ceb5ad..374c9423d2fc 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java @@ -788,7 +788,7 @@ public void testGetClusterState() throws Exception { }); // check volume, bucket and key count after running table count task - Pair> result = + Pair, Boolean>> result = omTableInsightTask.reprocess(reconOMMetadataManager); assertTrue(result.getRight().getRight()); response = clusterStateEndpoint.getClusterState(); @@ -867,7 +867,7 @@ public void testGetFileCounts() throws Exception { .thenReturn(omKeyInfo2) .thenReturn(omKeyInfo3); - Pair> result = + Pair, Boolean>> result = fileSizeCountTask.reprocess(omMetadataManager); assertTrue(result.getRight().getRight()); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java index d65e02d36d78..15d59448e88a 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java @@ -20,6 +20,8 @@ import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -54,21 +56,21 @@ public Collection getTaskTables() { } @Override - public Pair> process( - OMUpdateEventBatch events, int seekPos) { + public Pair, Boolean>> process( + OMUpdateEventBatch events, Map seekPos) { if (++callCtr <= numFailuresAllowed) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } else { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } } @Override - public Pair> reprocess(OMMetadataManager omMetadataManager) { + public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { if (++callCtr <= numFailuresAllowed) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, false)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); } else { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true)); + return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); } } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java index f9ce3005d145..8d37c6fdc4ff 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.nio.file.Path; +import java.util.HashMap; import java.util.Iterator; import java.util.ArrayList; import java.util.Collections; @@ -339,7 +340,7 @@ public void testKeyTableProcess() throws IOException { assertEquals(1, reconContainerMetadataManager.getKeyCountForContainer(3L)); // Process PUT & DELETE event. - containerKeyMapperTask.process(omUpdateEventBatch, 0); + containerKeyMapperTask.process(omUpdateEventBatch, new HashMap<>()); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); @@ -430,7 +431,7 @@ public void testFileTableProcess() throws Exception { }, 0L); // Process PUT event for both the keys - containerKeyMapperTask.process(omUpdateEventBatch, 0); + containerKeyMapperTask.process(omUpdateEventBatch, new HashMap<>()); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); @@ -463,7 +464,7 @@ public void testFileTableProcess() throws Exception { }, 0L); // Process DELETE event for key2 - containerKeyMapperTask.process(omUpdateEventBatch2, 0); + containerKeyMapperTask.process(omUpdateEventBatch2, new HashMap<>()); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java index b60cc08ed324..93af4ebfb29d 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java @@ -37,7 +37,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.DELETE; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.PUT; @@ -137,7 +139,7 @@ public void testReprocess() throws IOException { fileCountBySizeDao.insert( new FileCountBySize("vol1", "bucket1", 1024L, 10L)); - Pair> result = + Pair, Boolean>> result = fileSizeCountTask.reprocess(omMetadataManager); // Verify that the result of reprocess is true @@ -196,7 +198,7 @@ public void testProcess() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(Arrays.asList(event, event2), 0L); - fileSizeCountTask.process(omUpdateEventBatch, 0); + fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); // Verify 2 keys are in correct bins. assertEquals(2, fileCountBySizeDao.count()); @@ -251,7 +253,7 @@ public void testProcess() { omUpdateEventBatch = new OMUpdateEventBatch( Arrays.asList(updateEvent, putEvent, deleteEvent), 0L); - fileSizeCountTask.process(omUpdateEventBatch, 0); + fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); assertEquals(4, fileCountBySizeDao.count()); recordToFind.value3(1024L); @@ -324,7 +326,7 @@ public void testReprocessAtScale() throws IOException { when(mockKeyValueFso.getValue()) .thenAnswer(returnsElementsOf(omKeyInfoList)); - Pair> result = + Pair, Boolean>> result = fileSizeCountTask.reprocess(omMetadataManager); assertTrue(result.getRight().getRight()); @@ -389,7 +391,7 @@ public void testProcessAtScale() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList, 0L); - fileSizeCountTask.process(omUpdateEventBatch, 0); + fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); // Verify 2 keys are in correct bins. assertEquals(10000, fileCountBySizeDao.count()); @@ -465,7 +467,7 @@ public void testProcessAtScale() { } omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList, 0L); - fileSizeCountTask.process(omUpdateEventBatch, 0); + fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); assertEquals(10000, fileCountBySizeDao.count()); recordToFind = dslContext diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java index 76c6bb9b4299..c8507ebf99f5 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java @@ -44,6 +44,7 @@ import java.io.File; import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; import java.util.Set; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; @@ -223,7 +224,7 @@ public class TestProcess { @BeforeEach public void setUp() throws IOException { nSSummaryTask.reprocess(reconOMMetadataManager); - nSSummaryTask.process(processEventBatch(), 0); + nSSummaryTask.process(processEventBatch(), new HashMap<>()); nsSummaryForBucket1 = reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); 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 210c439bce97..5c9f9b786861 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 @@ -50,7 +50,9 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.BUCKET_TABLE; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE; @@ -281,7 +283,7 @@ public void testReprocessForDeletedDirectory() throws Exception { // Generate NamespaceSummary for the OM DB nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); - Pair> result = + Pair, Boolean>> result = omTableInsightTask.reprocess(reconOMMetadataManager); assertTrue(result.getRight().getRight()); assertEquals(3, getCountForTable(DELETED_DIR_TABLE)); @@ -321,7 +323,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { DELETED_DIR_TABLE, PUT, null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch, 0); + omTableInsightTask.process(putEventBatch, new HashMap<>()); assertEquals(5, getCountForTable(DELETED_DIR_TABLE)); @@ -335,7 +337,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { getOmKeyInfo("vol1", "bucket1", DIR_ONE, 3L, false), DELETED_DIR_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch, 0); + omTableInsightTask.process(deleteEventBatch, new HashMap<>()); assertEquals(3, getCountForTable(DELETED_DIR_TABLE)); } @@ -368,7 +370,7 @@ public void testReprocessForCount() throws Exception { when(mockIter.next()).thenReturn(mockKeyValue); } - Pair> result = + Pair, Boolean>> result = omTableInsightTask.reprocess(omMetadataManager); assertTrue(result.getRight().getRight()); @@ -389,7 +391,7 @@ public void testReprocessForOpenKeyTable() throws Exception { writeOpenKeyToOm(reconOMMetadataManager, "key1", "Bucket3", "Volume3", null, 3L); - Pair> result = + Pair, Boolean>> result = omTableInsightTask.reprocess(reconOMMetadataManager); assertTrue(result.getRight().getRight()); assertEquals(3L, getCountForTable(OPEN_KEY_TABLE)); @@ -408,7 +410,7 @@ public void testReprocessForOpenFileTable() throws Exception { writeOpenFileToOm(reconOMMetadataManager, "file3", "Bucket3", "Volume3", "file3", 3, 0, 3, 3, null, 3L); - Pair> result = + Pair, Boolean>> result = omTableInsightTask.reprocess(reconOMMetadataManager); assertTrue(result.getRight().getRight()); assertEquals(3L, getCountForTable(OPEN_FILE_TABLE)); @@ -432,7 +434,7 @@ public void testReprocessForDeletedTable() throws Exception { deletedKeysList3, "Bucket3", "Volume3"); - Pair> result = + Pair, Boolean>> result = omTableInsightTask.reprocess(reconOMMetadataManager); assertTrue(result.getRight().getRight()); assertEquals(6L, getCountForTable(DELETED_TABLE)); @@ -471,7 +473,7 @@ public void testProcessForCount() { // Processing the initial batch of events OMUpdateEventBatch initialBatch = new OMUpdateEventBatch(initialEvents, 0L); - omTableInsightTask.process(initialBatch, 0); + omTableInsightTask.process(initialBatch, new HashMap<>()); // Verifying the count in each table for (String tableName : omTableInsightTask.getTaskTables()) { @@ -500,7 +502,7 @@ public void testProcessForCount() { // Processing the additional events OMUpdateEventBatch additionalBatch = new OMUpdateEventBatch(additionalEvents, 0L); - omTableInsightTask.process(additionalBatch, 0); + omTableInsightTask.process(additionalBatch, new HashMap<>()); // Verifying the final count in each table for (String tableName : omTableInsightTask.getTaskTables()) { if (tableName.equals(DELETED_TABLE)) { @@ -529,7 +531,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch, 0); + omTableInsightTask.process(putEventBatch, new HashMap<>()); // After 5 PUTs, size should be 5 * 1000 = 5000 for (String tableName : new ArrayList<>( @@ -547,7 +549,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { getOMUpdateEvent("item0", omKeyInfo, OPEN_FILE_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch, 0); + omTableInsightTask.process(deleteEventBatch, new HashMap<>()); // After deleting "item0", size should be 4 * 1000 = 4000 for (String tableName : new ArrayList<>( @@ -570,7 +572,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch updateEventBatch = new OMUpdateEventBatch(updateEvents, 0L); - omTableInsightTask.process(updateEventBatch, 0); + omTableInsightTask.process(updateEventBatch, new HashMap<>()); // After updating "item1", size should be 4000 - 1000 + 2000 = 5000 // presentValue - oldValue + newValue = updatedValue @@ -607,7 +609,7 @@ public void testProcessForDeletedTable() { null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch, 0); + omTableInsightTask.process(putEventBatch, new HashMap<>()); // Each of the 5 RepeatedOmKeyInfo object has 5 OmKeyInfo obj, // so total deleted keys should be 5 * 5 = 25 assertEquals(25L, getCountForTable(DELETED_TABLE)); @@ -623,7 +625,7 @@ public void testProcessForDeletedTable() { getOMUpdateEvent("item0", repeatedOmKeyInfo, DELETED_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch, 0); + omTableInsightTask.process(deleteEventBatch, new HashMap<>()); // After deleting "item0" total deleted keys should be 20 assertEquals(20L, getCountForTable(DELETED_TABLE)); // After deleting "item0", size should be 4 * 1000 = 4000 diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java index 30f50c56da65..eaf65011c18f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java @@ -22,13 +22,14 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertSame; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; import static org.mockito.Mockito.verify; +import java.util.HashMap; import java.util.HashSet; import org.apache.commons.lang3.tuple.ImmutablePair; @@ -84,8 +85,8 @@ public void testRegisterTask() { @Test public void testConsumeOMEvents() throws Exception { ReconOmTask reconOmTaskMock = getMockTask("MockTask"); - when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyInt())) - .thenReturn(new ImmutablePair<>("MockTask", new ImmutablePair<>(0, true))); + when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyMap())) + .thenReturn(new ImmutablePair<>("MockTask", new ImmutablePair<>(new HashMap<>(), true))); reconTaskController.registerTask(reconOmTaskMock); OMUpdateEventBatch omUpdateEventBatchMock = mock(OMUpdateEventBatch.class); when(omUpdateEventBatchMock.getLastSequenceNumber()).thenReturn(100L); @@ -97,7 +98,7 @@ public void testConsumeOMEvents() throws Exception { mock(OMMetadataManager.class)); verify(reconOmTaskMock, times(1)) - .process(any(), anyInt()); + .process(any(), anyMap()); long endTime = System.currentTimeMillis(); ReconTaskStatus reconTaskStatus = reconTaskStatusDao.findById("MockTask"); @@ -114,7 +115,7 @@ public void testTaskRecordsFailureOnException() throws Exception { OMUpdateEventBatch omUpdateEventBatchMock = mock(OMUpdateEventBatch.class); // Throw exception when trying to run task - when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyInt())) + when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyMap())) .thenThrow(new RuntimeException("Mock Failure")); reconTaskController.registerTask(reconOmTaskMock); when(omUpdateEventBatchMock.getLastSequenceNumber()).thenReturn(100L); @@ -126,7 +127,7 @@ public void testTaskRecordsFailureOnException() throws Exception { mock(OMMetadataManager.class)); verify(reconOmTaskMock, times(1)) - .process(any(), anyInt()); + .process(any(), anyMap()); long endTime = System.currentTimeMillis(); ReconTaskStatus reconTaskStatus = reconTaskStatusDao.findById("MockTask"); @@ -212,7 +213,7 @@ public void testReInitializeTasks() throws Exception { ReconOmTask reconOmTaskMock = getMockTask("MockTask2"); when(reconOmTaskMock.reprocess(omMetadataManagerMock)) - .thenReturn(new ImmutablePair<>("MockTask2", new ImmutablePair<>(0, true))); + .thenReturn(new ImmutablePair<>("MockTask2", new ImmutablePair<>(new HashMap<>(), true))); when(omMetadataManagerMock.getLastSequenceNumberFromDB() ).thenReturn(100L); From 97c303213a8b908436dc6225d9f6ab05684962f0 Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Sat, 8 Feb 2025 22:10:30 +0530 Subject: [PATCH 09/13] HDDS-12062. Fixing javadoc compatibility for java11, java 17, java21. --- .../apache/hadoop/ozone/recon/tasks/ReconOmTask.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java index de08c2f05cf1..3355e29536f7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java @@ -37,10 +37,14 @@ public interface ReconOmTask { /** * Processes a set of OM events on tables that the task is listening to. * - * @param events Set of events to be processed by the task. - * @param subTaskSeekPosMap Position from where to start iterating events of the event iterator for a given sub-task. - * @return A pair containing the task name mapped to a map of - * {@code sub-task name, its events iterator position} and a task success flag. + * @param events The batch of OM update events to be processed. + * @param subTaskSeekPosMap A map containing the position from where to start + * iterating events for each sub-task. + * @return A pair where: + * - The first element is the task name. + * - The second element is another pair containing: + * - A map of sub-task names to their respective event iterator positions. + * - A boolean indicating task success. */ Pair, Boolean>> process(OMUpdateEventBatch events, Map subTaskSeekPosMap); From 6be840a2ceeb113c4911fdc093c8d4cb34488d5b Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Sat, 8 Feb 2025 23:14:13 +0530 Subject: [PATCH 10/13] HDDS-12062. Fixing javadoc compatibility for java11, java 17, java21. --- .../apache/hadoop/ozone/recon/tasks/ReconOmTask.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java index 3355e29536f7..33cb6591ac14 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java @@ -50,10 +50,14 @@ Pair, Boolean>> process(OMUpdateEventBatch eve Map subTaskSeekPosMap); /** - * Process a on tables that the task is listening on. + * Reprocesses full entries in Recon OM RocksDB tables that the task is listening to. * - * @param omMetadataManager OM Metadata manager instance. - * @return Pair of task name -> map of , task success. + * @param omMetadataManager The OM Metadata Manager instance used for accessing metadata. + * @return A pair where: + * - The first element is the task name. + * - The second element is another pair containing: + * - A map of sub-task names to their respective seek positions. + * - A boolean indicating whether the task was successful. */ Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager); From e5aff5ccca6a30a1b9dcf71f8e9da98de26ba82f Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Mon, 10 Feb 2025 20:00:58 +0530 Subject: [PATCH 11/13] HDDS-12062. Fixing review comments. --- .../recon/tasks/ContainerKeyMapperTask.java | 43 +++++--- .../ozone/recon/tasks/FileSizeCountTask.java | 28 +++-- .../ozone/recon/tasks/NSSummaryTask.java | 31 ++++-- .../ozone/recon/tasks/OmTableInsightTask.java | 33 ++++-- .../hadoop/ozone/recon/tasks/ReconOmTask.java | 101 +++++++++++++++--- .../recon/tasks/ReconTaskControllerImpl.java | 16 +-- .../hadoop/ozone/recon/api/TestEndpoints.java | 10 +- .../ozone/recon/tasks/DummyReconDBTask.java | 27 +++-- .../recon/tasks/TestFileSizeCountTask.java | 10 +- .../recon/tasks/TestOmTableInsightTask.java | 22 ++-- .../tasks/TestReconTaskControllerImpl.java | 6 +- 11 files changed, 230 insertions(+), 97 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java index cbfcdfdad232..970bf2f098fa 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java @@ -35,8 +35,6 @@ import java.util.Map; import java.util.Set; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.db.RDBBatchOperation; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -85,7 +83,7 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager * (container, key) -> count to Recon Container DB. */ @Override - public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { + public TaskResult reprocess(OMMetadataManager omMetadataManager) { long omKeyCount = 0; // In-memory maps for fast look up and batch write @@ -121,7 +119,10 @@ public Pair, Boolean>> reprocess(OMMetadataMan containerKeyCountMap); if (!checkAndCallFlushToDB(containerKeyMap)) { LOG.error("Unable to flush containerKey information to the DB"); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } omKeyCount++; } @@ -134,7 +135,10 @@ public Pair, Boolean>> reprocess(OMMetadataMan containerKeyCountMap)) { LOG.error("Unable to flush Container Key Count and " + "remaining Container Key information to the DB"); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } LOG.debug("Completed 'reprocess' of ContainerKeyMapperTask."); @@ -145,9 +149,15 @@ public Pair, Boolean>> reprocess(OMMetadataMan } catch (IOException ioEx) { LOG.error("Unable to populate Container Key data in Recon DB. ", ioEx); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } private boolean flushAndCommitContainerKeyInfoToDB( @@ -192,8 +202,8 @@ public Collection getTaskTables() { } @Override - public Pair, Boolean>> process(OMUpdateEventBatch events, - Map subTaskSeekPosMap) { + public TaskResult process(OMUpdateEventBatch events, + Map subTaskSeekPosMap) { Iterator eventIterator = events.getIterator(); int eventCount = 0; final Collection taskTables = getTaskTables(); @@ -250,18 +260,27 @@ public Pair, Boolean>> process(OMUpdateEventBa } catch (IOException e) { LOG.error("Unexpected exception while updating key data : {} ", updatedKey, e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } } try { writeToTheDB(containerKeyMap, containerKeyCountMap, deletedKeyCountList); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } LOG.debug("{} successfully processed {} OM DB update event(s) in {} milliseconds.", getTaskName(), eventCount, (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } private void writeToTheDB(Map containerKeyMap, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java index 9d0c33bd94f3..73995aba7903 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java @@ -19,8 +19,6 @@ package org.apache.hadoop.ozone.recon.tasks; import com.google.inject.Inject; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; @@ -75,7 +73,7 @@ public FileSizeCountTask(FileCountBySizeDao fileCountBySizeDao, * @return Pair */ @Override - public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { + public TaskResult reprocess(OMMetadataManager omMetadataManager) { // Map to store the count of files based on file size Map fileSizeCountMap = new HashMap<>(); @@ -93,11 +91,17 @@ public Pair, Boolean>> reprocess(OMMetadataMan reprocessBucketLayout(BucketLayout.LEGACY, omMetadataManager, fileSizeCountMap); if (!statusFSO && !statusOBS) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } writeCountsToDB(true, fileSizeCountMap); LOG.debug("Completed a 'reprocess' run of FileSizeCountTask."); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } private boolean reprocessBucketLayout(BucketLayout bucketLayout, @@ -145,8 +149,8 @@ public Collection getTaskTables() { * @return Pair */ @Override - public Pair, Boolean>> process(OMUpdateEventBatch events, - Map subTaskSeekPosMap) { + public TaskResult process(OMUpdateEventBatch events, + Map subTaskSeekPosMap) { Iterator eventIterator = events.getIterator(); Map fileSizeCountMap = new HashMap<>(); final Collection taskTables = getTaskTables(); @@ -193,7 +197,10 @@ public Pair, Boolean>> process(OMUpdateEventBa } catch (Exception e) { LOG.error("Unexpected exception while processing key {}.", updatedKey, e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } } else { LOG.warn("Unexpected value type {} for key {}. Skipping processing.", @@ -203,7 +210,10 @@ public Pair, Boolean>> process(OMUpdateEventBa writeCountsToDB(false, fileSizeCountMap); LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } /** 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 6b16b902b1a8..e0a7a6e2fc55 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 @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.recon.tasks; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.om.OMMetadataManager; @@ -125,7 +124,7 @@ public String getDescription() { } @Override - public Pair, Boolean>> process( + public TaskResult process( OMUpdateEventBatch events, Map subTaskSeekPosMap) { boolean anyFailure = false; // Track if any bucket fails Map updatedSeekPositions = new HashMap<>(); @@ -158,11 +157,15 @@ public Pair, Boolean>> process( } // Return task failure if any bucket failed, while keeping each bucket's latest seek position - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(updatedSeekPositions, !anyFailure)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setSubTaskSeekPositions(updatedSeekPositions) + .setTaskSuccess(!anyFailure) + .build(); } @Override - public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { + public TaskResult reprocess(OMMetadataManager omMetadataManager) { // Initialize a list of tasks to run in parallel Collection> tasks = new ArrayList<>(); @@ -174,7 +177,10 @@ public Pair, Boolean>> reprocess(OMMetadataMan } catch (IOException ioEx) { LOG.error("Unable to clear NSSummary table in Recon DB. ", ioEx); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } tasks.add(() -> nsSummaryTaskWithFSO @@ -194,12 +200,18 @@ public Pair, Boolean>> reprocess(OMMetadataMan results = executorService.invokeAll(tasks); for (int i = 0; i < results.size(); i++) { if (results.get(i).get().equals(false)) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } } } catch (InterruptedException | ExecutionException ex) { LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } finally { executorService.shutdown(); @@ -212,7 +224,10 @@ public Pair, Boolean>> reprocess(OMMetadataMan LOG.debug("Task execution time: {} milliseconds", durationInMillis); } - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } } 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 1e55b74f5d1f..3e98ff6158dc 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 @@ -20,8 +20,6 @@ import com.google.common.collect.Iterators; import com.google.inject.Inject; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; @@ -91,7 +89,7 @@ public OmTableInsightTask(GlobalStatsDao globalStatsDao, * @return Pair */ @Override - public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { + public TaskResult reprocess(OMMetadataManager omMetadataManager) { HashMap objectCountMap = initializeCountMap(); HashMap unReplicatedSizeMap = initializeSizeMap(false); HashMap replicatedSizeMap = initializeSizeMap(true); @@ -100,7 +98,10 @@ public Pair, Boolean>> reprocess(OMMetadataMan Table table = omMetadataManager.getTable(tableName); if (table == null) { LOG.error("Table " + tableName + " not found in OM Metadata."); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } try (TableIterator> iterator @@ -120,7 +121,10 @@ public Pair, Boolean>> reprocess(OMMetadataMan } } catch (IOException ioEx) { LOG.error("Unable to populate Table Count in Recon DB.", ioEx); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } } // Write the data to the DB @@ -135,7 +139,10 @@ public Pair, Boolean>> reprocess(OMMetadataMan } LOG.debug("Completed a 'reprocess' run of OmTableInsightTask."); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } @Override @@ -156,8 +163,8 @@ public Collection getTaskTables() { * @return Pair */ @Override - public Pair, Boolean>> process(OMUpdateEventBatch events, - Map subTaskSeekPosMap) { + public TaskResult process(OMUpdateEventBatch events, + Map subTaskSeekPosMap) { Iterator eventIterator = events.getIterator(); // Initialize maps to store count and size information HashMap objectCountMap = initializeCountMap(); @@ -198,7 +205,10 @@ public Pair, Boolean>> process(OMUpdateEventBa LOG.error( "Unexpected exception while processing the table {}, Action: {}", tableName, omdbUpdateEvent.getAction(), e); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } } // Write the updated count and size information to the database @@ -213,7 +223,10 @@ public Pair, Boolean>> process(OMUpdateEventBa } LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } private void handlePutEvent(OMDBUpdateEvent event, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java index 33cb6591ac14..67d3257817c7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java @@ -18,9 +18,9 @@ package org.apache.hadoop.ozone.recon.tasks; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.ozone.om.OMMetadataManager; +import java.util.Collections; import java.util.Map; /** @@ -40,25 +40,98 @@ public interface ReconOmTask { * @param events The batch of OM update events to be processed. * @param subTaskSeekPosMap A map containing the position from where to start * iterating events for each sub-task. - * @return A pair where: - * - The first element is the task name. - * - The second element is another pair containing: - * - A map of sub-task names to their respective event iterator positions. - * - A boolean indicating task success. + * @return A {@link TaskResult} containing: + * - The task name. + * - A map of sub-task names to their respective seek positions. + * - A boolean indicating whether the task was successful. */ - Pair, Boolean>> process(OMUpdateEventBatch events, - Map subTaskSeekPosMap); + TaskResult process(OMUpdateEventBatch events, + Map subTaskSeekPosMap); /** * Reprocesses full entries in Recon OM RocksDB tables that the task is listening to. * * @param omMetadataManager The OM Metadata Manager instance used for accessing metadata. - * @return A pair where: - * - The first element is the task name. - * - The second element is another pair containing: - * - A map of sub-task names to their respective seek positions. - * - A boolean indicating whether the task was successful. + * @return A {@link TaskResult} containing: + * - The task name. + * - A map of sub-task names to their respective seek positions. + * - A boolean indicating whether the task was successful. */ - Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager); + TaskResult reprocess(OMMetadataManager omMetadataManager); + /** + * Represents the result of a task execution, including the task name, + * sub-task seek positions, and success status. + * + *

This class is immutable and uses the Builder pattern for object creation.

+ */ + class TaskResult { + private final String taskName; + private final Map subTaskSeekPositions; + private final boolean taskSuccess; + + /** + * Private constructor to enforce the use of the {@link Builder}. + * + * @param builder The builder instance containing values for initialization. + */ + private TaskResult(Builder builder) { + this.taskName = builder.taskName; + this.subTaskSeekPositions = builder.subTaskSeekPositions != null + ? builder.subTaskSeekPositions + : Collections.emptyMap(); // Default value + this.taskSuccess = builder.taskSuccess; + } + + // Getters + public String getTaskName() { + return taskName; + } + + public Map getSubTaskSeekPositions() { + return subTaskSeekPositions; + } + + public boolean isTaskSuccess() { + return taskSuccess; + } + + /** + * Builder class for creating instances of {@link TaskResult}. + */ + public static class Builder { + private String taskName; + private Map subTaskSeekPositions = Collections.emptyMap(); // Default value + private boolean taskSuccess; + + public Builder setTaskName(String taskName) { + this.taskName = taskName; + return this; + } + + public Builder setSubTaskSeekPositions(Map subTaskSeekPositions) { + this.subTaskSeekPositions = subTaskSeekPositions; + return this; + } + + public Builder setTaskSuccess(boolean taskSuccess) { + this.taskSuccess = taskSuccess; + return this; + } + + public TaskResult build() { + return new TaskResult(this); + } + } + + // toString Method for debugging + @Override + public String toString() { + return "TaskResult{" + + "taskName='" + taskName + '\'' + + ", subTaskSeekPositions=" + subTaskSeekPositions + + ", taskSuccess=" + taskSuccess + + '}'; + } + } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java index 4868b51df81b..48d578a787a5 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java @@ -100,7 +100,7 @@ public void registerTask(ReconOmTask task) { @Override public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataManager omMetadataManager) { if (!events.isEmpty()) { - Collection, Boolean>>>> tasks = new ArrayList<>(); + Collection> tasks = new ArrayList<>(); List>> failedTasks = new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { @@ -161,7 +161,7 @@ private void ignoreFailedTasks(List>> failedTa @Override public synchronized void reInitializeTasks(ReconOMMetadataManager omMetadataManager) { - Collection, Boolean>>>> tasks = new ArrayList<>(); + Collection> tasks = new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { ReconOmTask task = taskEntry.getValue(); @@ -185,10 +185,10 @@ public synchronized void reInitializeTasks(ReconOMMetadataManager omMetadataMana throw new TaskExecutionException(task.getTaskName(), e); } }, executorService).thenAccept(result -> { - String taskName = result.getLeft(); + String taskName = result.getTaskName(); ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(taskName); - if (!result.getRight().getRight()) { + if (!result.isTaskSuccess()) { LOG.error("Init failed for task {}.", taskName); taskStatusUpdater.setLastTaskRunStatus(-1); } else { @@ -247,7 +247,7 @@ public synchronized void stop() { * @param failedTasks Reference of the list to which we want to add the failed tasks for retry/reprocessing */ private void processTasks( - Collection, Boolean>>>> tasks, + Collection> tasks, OMUpdateEventBatch events, List>> failedTasks) { List> futures = tasks.stream() .map(task -> CompletableFuture.supplyAsync(() -> { @@ -261,12 +261,12 @@ private void processTasks( throw new TaskExecutionException(task.getTaskName(), e); } }, executorService).thenAccept(result -> { - String taskName = result.getLeft(); + String taskName = result.getTaskName(); ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(taskName); - if (!result.getRight().getRight()) { + if (!result.isTaskSuccess()) { LOG.error("Task {} failed", taskName); - failedTasks.add(new ImmutablePair<>(taskName, result.getRight().getLeft())); + failedTasks.add(new ImmutablePair<>(taskName, result.getSubTaskSeekPositions())); taskStatusUpdater.setLastTaskRunStatus(-1); } else { taskFailureCounter.get(taskName).set(0); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java index 374c9423d2fc..d42778f3c75a 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestEndpoints.java @@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.recon.api; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -89,6 +88,7 @@ import org.apache.hadoop.ozone.recon.tasks.ContainerSizeCountTask; import org.apache.hadoop.ozone.recon.tasks.FileSizeCountTask; import org.apache.hadoop.ozone.recon.tasks.OmTableInsightTask; +import org.apache.hadoop.ozone.recon.tasks.ReconOmTask; import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; import org.apache.ozone.test.LambdaTestUtils; import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition; @@ -788,9 +788,9 @@ public void testGetClusterState() throws Exception { }); // check volume, bucket and key count after running table count task - Pair, Boolean>> result = + ReconOmTask.TaskResult result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); response = clusterStateEndpoint.getClusterState(); clusterStateResponse = (ClusterStateResponse) response.getEntity(); assertEquals(2, clusterStateResponse.getVolumes()); @@ -867,9 +867,9 @@ public void testGetFileCounts() throws Exception { .thenReturn(omKeyInfo2) .thenReturn(omKeyInfo3); - Pair, Boolean>> result = + ReconOmTask.TaskResult result = fileSizeCountTask.reprocess(omMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); assertEquals(3, fileCountBySizeDao.count()); Response response = utilizationEndpoint.getFileCounts(null, null, 0); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java index 15d59448e88a..c3a785b19a23 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java @@ -20,11 +20,8 @@ import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Map; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.ozone.om.OMMetadataManager; /** @@ -56,21 +53,33 @@ public Collection getTaskTables() { } @Override - public Pair, Boolean>> process( + public TaskResult process( OMUpdateEventBatch events, Map seekPos) { if (++callCtr <= numFailuresAllowed) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } else { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } } @Override - public Pair, Boolean>> reprocess(OMMetadataManager omMetadataManager) { + public TaskResult reprocess(OMMetadataManager omMetadataManager) { if (++callCtr <= numFailuresAllowed) { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), false)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(false) + .build(); } else { - return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(new HashMap<>(), true)); + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(true) + .build(); } } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java index 93af4ebfb29d..226b177f3df0 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.recon.tasks; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.helpers.BucketLayout; @@ -39,7 +38,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; -import java.util.Map; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.DELETE; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.PUT; @@ -139,11 +137,11 @@ public void testReprocess() throws IOException { fileCountBySizeDao.insert( new FileCountBySize("vol1", "bucket1", 1024L, 10L)); - Pair, Boolean>> result = + ReconOmTask.TaskResult result = fileSizeCountTask.reprocess(omMetadataManager); // Verify that the result of reprocess is true - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); // Verify that the number of entries in fileCountBySizeDao is 3 assertEquals(3, fileCountBySizeDao.count()); @@ -326,9 +324,9 @@ public void testReprocessAtScale() throws IOException { when(mockKeyValueFso.getValue()) .thenAnswer(returnsElementsOf(omKeyInfoList)); - Pair, Boolean>> result = + ReconOmTask.TaskResult result = fileSizeCountTask.reprocess(omMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); // 2 volumes * 500 buckets * 42 bins = 42000 rows assertEquals(42000, fileCountBySizeDao.count()); 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 5c9f9b786861..2ffd315e0830 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 @@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.recon.tasks; import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -52,7 +51,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; -import java.util.Map; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.BUCKET_TABLE; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE; @@ -283,9 +281,9 @@ public void testReprocessForDeletedDirectory() throws Exception { // Generate NamespaceSummary for the OM DB nSSummaryTaskWithFso.reprocessWithFSO(reconOMMetadataManager); - Pair, Boolean>> result = + ReconOmTask.TaskResult result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); assertEquals(3, getCountForTable(DELETED_DIR_TABLE)); } @@ -370,10 +368,10 @@ public void testReprocessForCount() throws Exception { when(mockIter.next()).thenReturn(mockKeyValue); } - Pair, Boolean>> result = + ReconOmTask.TaskResult result = omTableInsightTask.reprocess(omMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); assertEquals(5L, getCountForTable(KEY_TABLE)); assertEquals(5L, getCountForTable(VOLUME_TABLE)); assertEquals(5L, getCountForTable(BUCKET_TABLE)); @@ -391,9 +389,9 @@ public void testReprocessForOpenKeyTable() throws Exception { writeOpenKeyToOm(reconOMMetadataManager, "key1", "Bucket3", "Volume3", null, 3L); - Pair, Boolean>> result = + ReconOmTask.TaskResult result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); assertEquals(3L, getCountForTable(OPEN_KEY_TABLE)); // Test for both replicated and unreplicated size for OPEN_KEY_TABLE assertEquals(6L, getUnReplicatedSizeForTable(OPEN_KEY_TABLE)); @@ -410,9 +408,9 @@ public void testReprocessForOpenFileTable() throws Exception { writeOpenFileToOm(reconOMMetadataManager, "file3", "Bucket3", "Volume3", "file3", 3, 0, 3, 3, null, 3L); - Pair, Boolean>> result = + ReconOmTask.TaskResult result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); assertEquals(3L, getCountForTable(OPEN_FILE_TABLE)); // Test for both replicated and unreplicated size for OPEN_FILE_TABLE assertEquals(6L, getUnReplicatedSizeForTable(OPEN_FILE_TABLE)); @@ -434,9 +432,9 @@ public void testReprocessForDeletedTable() throws Exception { deletedKeysList3, "Bucket3", "Volume3"); - Pair, Boolean>> result = + ReconOmTask.TaskResult result = omTableInsightTask.reprocess(reconOMMetadataManager); - assertTrue(result.getRight().getRight()); + assertTrue(result.isTaskSuccess()); assertEquals(6L, getCountForTable(DELETED_TABLE)); // Test for both replicated and unreplicated size for DELETED_TABLE assertEquals(600L, getUnReplicatedSizeForTable(DELETED_TABLE)); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java index eaf65011c18f..9283e132ff4c 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestReconTaskControllerImpl.java @@ -29,10 +29,8 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.verify; -import java.util.HashMap; import java.util.HashSet; -import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.recon.persistence.AbstractReconSqlDBTest; @@ -86,7 +84,7 @@ public void testRegisterTask() { public void testConsumeOMEvents() throws Exception { ReconOmTask reconOmTaskMock = getMockTask("MockTask"); when(reconOmTaskMock.process(any(OMUpdateEventBatch.class), anyMap())) - .thenReturn(new ImmutablePair<>("MockTask", new ImmutablePair<>(new HashMap<>(), true))); + .thenReturn(new ReconOmTask.TaskResult.Builder().setTaskName("MockTask").setTaskSuccess(true).build()); reconTaskController.registerTask(reconOmTaskMock); OMUpdateEventBatch omUpdateEventBatchMock = mock(OMUpdateEventBatch.class); when(omUpdateEventBatchMock.getLastSequenceNumber()).thenReturn(100L); @@ -213,7 +211,7 @@ public void testReInitializeTasks() throws Exception { ReconOmTask reconOmTaskMock = getMockTask("MockTask2"); when(reconOmTaskMock.reprocess(omMetadataManagerMock)) - .thenReturn(new ImmutablePair<>("MockTask2", new ImmutablePair<>(new HashMap<>(), true))); + .thenReturn(new ReconOmTask.TaskResult.Builder().setTaskName("MockTask2").setTaskSuccess(true).build()); when(omMetadataManagerMock.getLastSequenceNumberFromDB() ).thenReturn(100L); From 1d5ecda3d68aadebb8919120b8cbdfa95c4b4f22 Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Wed, 12 Feb 2025 14:36:42 +0530 Subject: [PATCH 12/13] HDDS-12062. Fixing review comments. --- .../recon/tasks/ReconTaskControllerImpl.java | 34 ++++++++++--------- .../tasks/TestContainerKeyMapperTask.java | 7 ++-- .../recon/tasks/TestFileSizeCountTask.java | 10 +++--- .../ozone/recon/tasks/TestNSSummaryTask.java | 4 +-- .../recon/tasks/TestOmTableInsightTask.java | 20 +++++------ 5 files changed, 38 insertions(+), 37 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java index 48d578a787a5..ebb650f00817 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -37,8 +38,6 @@ import java.util.stream.Collectors; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; @@ -101,26 +100,26 @@ public void registerTask(ReconOmTask task) { public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataManager omMetadataManager) { if (!events.isEmpty()) { Collection> tasks = new ArrayList<>(); - List>> failedTasks = new ArrayList<>(); + List failedTasks = new ArrayList<>(); for (Map.Entry taskEntry : reconOmTasks.entrySet()) { ReconOmTask task = taskEntry.getValue(); ReconTaskStatusUpdater taskStatusUpdater = taskStatusUpdaterManager.getTaskStatusUpdater(task.getTaskName()); taskStatusUpdater.recordRunStart(); // events passed to process method is no longer filtered - tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.process(events, new HashMap<>()))); + tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.process(events, Collections.emptyMap()))); } processTasks(tasks, events, failedTasks); // Retry processing failed tasks - List>> retryFailedTasks = new ArrayList<>(); + List retryFailedTasks = new ArrayList<>(); if (!failedTasks.isEmpty()) { tasks.clear(); - for (Pair> taskPair : failedTasks) { - ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); + for (ReconOmTask.TaskResult taskResult : failedTasks) { + ReconOmTask task = reconOmTasks.get(taskResult.getTaskName()); // events passed to process method is no longer filtered tasks.add(new NamedCallableTask<>(task.getTaskName(), - () -> task.process(events, taskPair.getRight()))); + () -> task.process(events, taskResult.getSubTaskSeekPositions()))); } processTasks(tasks, events, retryFailedTasks); } @@ -128,11 +127,11 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataMa // Reprocess the failed tasks. if (!retryFailedTasks.isEmpty()) { tasks.clear(); - for (Pair> taskPair : failedTasks) { - ReconOmTask task = reconOmTasks.get(taskPair.getLeft()); + for (ReconOmTask.TaskResult taskResult : failedTasks) { + ReconOmTask task = reconOmTasks.get(taskResult.getTaskName()); tasks.add(new NamedCallableTask<>(task.getTaskName(), () -> task.reprocess(omMetadataManager))); } - List>> reprocessFailedTasks = new ArrayList<>(); + List reprocessFailedTasks = new ArrayList<>(); processTasks(tasks, events, reprocessFailedTasks); // Here the assumption is that even if full re-process of task also fails, // then there is something wrong in recon rocks DB got from OM and needs to be @@ -146,9 +145,9 @@ public synchronized void consumeOMEvents(OMUpdateEventBatch events, OMMetadataMa * Ignore tasks that failed reprocess step more than threshold times. * @param failedTasks list of failed tasks. */ - private void ignoreFailedTasks(List>> failedTasks) { - for (Pair> taskPair : failedTasks) { - String taskName = taskPair.getLeft(); + private void ignoreFailedTasks(List failedTasks) { + for (ReconOmTask.TaskResult taskResult : failedTasks) { + String taskName = taskResult.getTaskName(); LOG.info("Reprocess step failed for task {}.", taskName); if (taskFailureCounter.get(taskName).incrementAndGet() > TASK_FAILURE_THRESHOLD) { @@ -248,7 +247,7 @@ public synchronized void stop() { */ private void processTasks( Collection> tasks, - OMUpdateEventBatch events, List>> failedTasks) { + OMUpdateEventBatch events, List failedTasks) { List> futures = tasks.stream() .map(task -> CompletableFuture.supplyAsync(() -> { try { @@ -266,7 +265,10 @@ private void processTasks( taskStatusUpdaterManager.getTaskStatusUpdater(taskName); if (!result.isTaskSuccess()) { LOG.error("Task {} failed", taskName); - failedTasks.add(new ImmutablePair<>(taskName, result.getSubTaskSeekPositions())); + failedTasks.add(new ReconOmTask.TaskResult.Builder() + .setTaskName(taskName) + .setSubTaskSeekPositions(result.getSubTaskSeekPositions()) + .build()); taskStatusUpdater.setLastTaskRunStatus(-1); } else { taskFailureCounter.get(taskName).set(0); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java index 8d37c6fdc4ff..7dd7fb6978d3 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestContainerKeyMapperTask.java @@ -31,7 +31,6 @@ import java.io.IOException; import java.nio.file.Path; -import java.util.HashMap; import java.util.Iterator; import java.util.ArrayList; import java.util.Collections; @@ -340,7 +339,7 @@ public void testKeyTableProcess() throws IOException { assertEquals(1, reconContainerMetadataManager.getKeyCountForContainer(3L)); // Process PUT & DELETE event. - containerKeyMapperTask.process(omUpdateEventBatch, new HashMap<>()); + containerKeyMapperTask.process(omUpdateEventBatch, Collections.emptyMap()); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); @@ -431,7 +430,7 @@ public void testFileTableProcess() throws Exception { }, 0L); // Process PUT event for both the keys - containerKeyMapperTask.process(omUpdateEventBatch, new HashMap<>()); + containerKeyMapperTask.process(omUpdateEventBatch, Collections.emptyMap()); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); @@ -464,7 +463,7 @@ public void testFileTableProcess() throws Exception { }, 0L); // Process DELETE event for key2 - containerKeyMapperTask.process(omUpdateEventBatch2, new HashMap<>()); + containerKeyMapperTask.process(omUpdateEventBatch2, Collections.emptyMap()); keyPrefixesForContainer = reconContainerMetadataManager .getKeyPrefixesForContainer(1); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java index 226b177f3df0..0e5f6ddf6558 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestFileSizeCountTask.java @@ -36,7 +36,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import static org.apache.hadoop.ozone.recon.tasks.OMDBUpdateEvent.OMDBUpdateAction.DELETE; @@ -196,7 +196,7 @@ public void testProcess() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(Arrays.asList(event, event2), 0L); - fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); + fileSizeCountTask.process(omUpdateEventBatch, Collections.emptyMap()); // Verify 2 keys are in correct bins. assertEquals(2, fileCountBySizeDao.count()); @@ -251,7 +251,7 @@ public void testProcess() { omUpdateEventBatch = new OMUpdateEventBatch( Arrays.asList(updateEvent, putEvent, deleteEvent), 0L); - fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); + fileSizeCountTask.process(omUpdateEventBatch, Collections.emptyMap()); assertEquals(4, fileCountBySizeDao.count()); recordToFind.value3(1024L); @@ -389,7 +389,7 @@ public void testProcessAtScale() { OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList, 0L); - fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); + fileSizeCountTask.process(omUpdateEventBatch, Collections.emptyMap()); // Verify 2 keys are in correct bins. assertEquals(10000, fileCountBySizeDao.count()); @@ -465,7 +465,7 @@ public void testProcessAtScale() { } omUpdateEventBatch = new OMUpdateEventBatch(omDbEventList, 0L); - fileSizeCountTask.process(omUpdateEventBatch, new HashMap<>()); + fileSizeCountTask.process(omUpdateEventBatch, Collections.emptyMap()); assertEquals(10000, fileCountBySizeDao.count()); recordToFind = dslContext diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java index c8507ebf99f5..ff3d691bf16b 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java @@ -44,7 +44,7 @@ import java.io.File; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; +import java.util.Collections; import java.util.Set; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; @@ -224,7 +224,7 @@ public class TestProcess { @BeforeEach public void setUp() throws IOException { nSSummaryTask.reprocess(reconOMMetadataManager); - nSSummaryTask.process(processEventBatch(), new HashMap<>()); + nSSummaryTask.process(processEventBatch(), Collections.emptyMap()); nsSummaryForBucket1 = reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID); 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 2ffd315e0830..7e12d41b419c 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 @@ -49,7 +49,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.BUCKET_TABLE; @@ -321,7 +321,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { DELETED_DIR_TABLE, PUT, null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch, new HashMap<>()); + omTableInsightTask.process(putEventBatch, Collections.emptyMap()); assertEquals(5, getCountForTable(DELETED_DIR_TABLE)); @@ -335,7 +335,7 @@ public void testProcessForDeletedDirectoryTable() throws IOException { getOmKeyInfo("vol1", "bucket1", DIR_ONE, 3L, false), DELETED_DIR_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch, new HashMap<>()); + omTableInsightTask.process(deleteEventBatch, Collections.emptyMap()); assertEquals(3, getCountForTable(DELETED_DIR_TABLE)); } @@ -471,7 +471,7 @@ public void testProcessForCount() { // Processing the initial batch of events OMUpdateEventBatch initialBatch = new OMUpdateEventBatch(initialEvents, 0L); - omTableInsightTask.process(initialBatch, new HashMap<>()); + omTableInsightTask.process(initialBatch, Collections.emptyMap()); // Verifying the count in each table for (String tableName : omTableInsightTask.getTaskTables()) { @@ -500,7 +500,7 @@ public void testProcessForCount() { // Processing the additional events OMUpdateEventBatch additionalBatch = new OMUpdateEventBatch(additionalEvents, 0L); - omTableInsightTask.process(additionalBatch, new HashMap<>()); + omTableInsightTask.process(additionalBatch, Collections.emptyMap()); // Verifying the final count in each table for (String tableName : omTableInsightTask.getTaskTables()) { if (tableName.equals(DELETED_TABLE)) { @@ -529,7 +529,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch, new HashMap<>()); + omTableInsightTask.process(putEventBatch, Collections.emptyMap()); // After 5 PUTs, size should be 5 * 1000 = 5000 for (String tableName : new ArrayList<>( @@ -547,7 +547,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { getOMUpdateEvent("item0", omKeyInfo, OPEN_FILE_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch, new HashMap<>()); + omTableInsightTask.process(deleteEventBatch, Collections.emptyMap()); // After deleting "item0", size should be 4 * 1000 = 4000 for (String tableName : new ArrayList<>( @@ -570,7 +570,7 @@ public void testProcessForOpenKeyTableAndOpenFileTable() { } OMUpdateEventBatch updateEventBatch = new OMUpdateEventBatch(updateEvents, 0L); - omTableInsightTask.process(updateEventBatch, new HashMap<>()); + omTableInsightTask.process(updateEventBatch, Collections.emptyMap()); // After updating "item1", size should be 4000 - 1000 + 2000 = 5000 // presentValue - oldValue + newValue = updatedValue @@ -607,7 +607,7 @@ public void testProcessForDeletedTable() { null)); } OMUpdateEventBatch putEventBatch = new OMUpdateEventBatch(putEvents, 0L); - omTableInsightTask.process(putEventBatch, new HashMap<>()); + omTableInsightTask.process(putEventBatch, Collections.emptyMap()); // Each of the 5 RepeatedOmKeyInfo object has 5 OmKeyInfo obj, // so total deleted keys should be 5 * 5 = 25 assertEquals(25L, getCountForTable(DELETED_TABLE)); @@ -623,7 +623,7 @@ public void testProcessForDeletedTable() { getOMUpdateEvent("item0", repeatedOmKeyInfo, DELETED_TABLE, DELETE, null)); OMUpdateEventBatch deleteEventBatch = new OMUpdateEventBatch(deleteEvents, 0L); - omTableInsightTask.process(deleteEventBatch, new HashMap<>()); + omTableInsightTask.process(deleteEventBatch, Collections.emptyMap()); // After deleting "item0" total deleted keys should be 20 assertEquals(20L, getCountForTable(DELETED_TABLE)); // After deleting "item0", size should be 4 * 1000 = 4000 From 9aaf2bef3f082ab2f91dc2e720c84e8685e40c5c Mon Sep 17 00:00:00 2001 From: deveshsingh Date: Wed, 19 Feb 2025 18:08:33 +0530 Subject: [PATCH 13/13] HDDS-12062. Fixed review comments to add a helper method. --- .../recon/tasks/ContainerKeyMapperTask.java | 35 ++++--------------- .../ozone/recon/tasks/FileSizeCountTask.java | 20 +++-------- .../ozone/recon/tasks/NSSummaryTask.java | 20 +++-------- .../ozone/recon/tasks/OmTableInsightTask.java | 20 +++-------- .../hadoop/ozone/recon/tasks/ReconOmTask.java | 7 ++++ .../ozone/recon/tasks/DummyReconDBTask.java | 20 +++-------- 6 files changed, 30 insertions(+), 92 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java index 2a805f087b8d..e42e021b9e45 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperTask.java @@ -116,10 +116,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { containerKeyCountMap); if (!checkAndCallFlushToDB(containerKeyMap)) { LOG.error("Unable to flush containerKey information to the DB"); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } omKeyCount++; } @@ -132,10 +129,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { containerKeyCountMap)) { LOG.error("Unable to flush Container Key Count and " + "remaining Container Key information to the DB"); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } LOG.debug("Completed 'reprocess' of ContainerKeyMapperTask."); @@ -146,15 +140,9 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { } catch (IOException ioEx) { LOG.error("Unable to populate Container Key data in Recon DB. ", ioEx); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } private boolean flushAndCommitContainerKeyInfoToDB( @@ -257,27 +245,18 @@ public TaskResult process(OMUpdateEventBatch events, } catch (IOException e) { LOG.error("Unexpected exception while updating key data : {} ", updatedKey, e); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } } try { writeToTheDB(containerKeyMap, containerKeyCountMap, deletedKeyCountList); } catch (IOException e) { LOG.error("Unable to write Container Key Prefix data in Recon DB.", e); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } LOG.debug("{} successfully processed {} OM DB update event(s) in {} milliseconds.", getTaskName(), eventCount, (System.currentTimeMillis() - startTime)); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } private void writeToTheDB(Map containerKeyMap, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java index 7f7b9345b133..3f4c52816be0 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTask.java @@ -92,17 +92,11 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { reprocessBucketLayout(BucketLayout.LEGACY, omMetadataManager, fileSizeCountMap); if (!statusFSO && !statusOBS) { - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } writeCountsToDB(fileSizeCountMap); LOG.debug("Completed a 'reprocess' run of FileSizeCountTask."); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } private boolean reprocessBucketLayout(BucketLayout bucketLayout, @@ -202,10 +196,7 @@ public TaskResult process(OMUpdateEventBatch events, } catch (Exception e) { LOG.error("Unexpected exception while processing key {}.", updatedKey, e); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } } else { LOG.warn("Unexpected value type {} for key {}. Skipping processing.", @@ -215,10 +206,7 @@ public TaskResult process(OMUpdateEventBatch events, writeCountsToDB(fileSizeCountMap); LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } /** diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java index 857c3fd36314..aa2a94caf1bf 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 @@ -177,10 +177,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { } catch (IOException ioEx) { LOG.error("Unable to clear NSSummary table in Recon DB. ", ioEx); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } tasks.add(() -> nsSummaryTaskWithFSO @@ -200,18 +197,12 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { results = executorService.invokeAll(tasks); for (int i = 0; i < results.size(); i++) { if (results.get(i).get().equals(false)) { - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } } } catch (InterruptedException | ExecutionException ex) { LOG.error("Error while reprocessing NSSummary table in Recon DB.", ex); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } finally { executorService.shutdown(); @@ -224,10 +215,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { LOG.debug("Task execution time: {} milliseconds", durationInMillis); } - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java index 114f782cdbb2..6019990158bf 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 @@ -129,10 +129,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { } } catch (IOException ioEx) { LOG.error("Unable to populate Table Count in Recon DB.", ioEx); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } } // Write the data to the DB @@ -147,10 +144,7 @@ public TaskResult reprocess(OMMetadataManager omMetadataManager) { } LOG.debug("Completed a 'reprocess' run of OmTableInsightTask."); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } @Override @@ -207,10 +201,7 @@ public TaskResult process(OMUpdateEventBatch events, LOG.error( "Unexpected exception while processing the table {}, Action: {}", tableName, omdbUpdateEvent.getAction(), e); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } } // Write the updated count and size information to the database @@ -225,10 +216,7 @@ public TaskResult process(OMUpdateEventBatch events, } LOG.debug("{} successfully processed in {} milliseconds", getTaskName(), (System.currentTimeMillis() - startTime)); - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } private void handlePutEvent(OMDBUpdateEvent event, diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java index f7e483599ffe..395fdf6b1e00 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconOmTask.java @@ -137,4 +137,11 @@ public String toString() { '}'; } } + + default TaskResult buildTaskResult(boolean success) { + return new TaskResult.Builder() + .setTaskName(getTaskName()) + .setTaskSuccess(success) + .build(); + } } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java index c6ccf4d07dbb..2fc1c320d18b 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/DummyReconDBTask.java @@ -54,30 +54,18 @@ public Collection getTaskTables() { public TaskResult process( OMUpdateEventBatch events, Map seekPos) { if (++callCtr <= numFailuresAllowed) { - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } else { - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } } @Override public TaskResult reprocess(OMMetadataManager omMetadataManager) { if (++callCtr <= numFailuresAllowed) { - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(false) - .build(); + return buildTaskResult(false); } else { - return new TaskResult.Builder() - .setTaskName(getTaskName()) - .setTaskSuccess(true) - .build(); + return buildTaskResult(true); } }