Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-12062. Recon - Error handling in NSSummaryTask to avoid data inconsistencies. #7723

Open
wants to merge 8 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -510,7 +510,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()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager
* (container, key) -> count to Recon Container DB.
*/
@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
public Pair<String, Pair<Integer, Boolean>> reprocess(OMMetadataManager omMetadataManager) {
long omKeyCount = 0;

// In-memory maps for fast look up and batch write
Expand Down Expand Up @@ -121,7 +121,7 @@ public Pair<String, Boolean> 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++;
}
Expand All @@ -134,7 +134,7 @@ public Pair<String, Boolean> 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.");
Expand All @@ -145,9 +145,9 @@ public Pair<String, Boolean> 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(
Expand Down Expand Up @@ -192,7 +192,8 @@ public Collection<String> getTaskTables() {
}

@Override
public Pair<String, Boolean> process(OMUpdateEventBatch events) {
public Pair<String, Pair<Integer, Boolean>> process(OMUpdateEventBatch events,
int seekPosition) {
Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
int eventCount = 0;
final Collection<String> taskTables = getTaskTables();
Expand Down Expand Up @@ -249,18 +250,18 @@ public Pair<String, Boolean> 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<ContainerKeyPrefix, Integer> containerKeyMap,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public FileSizeCountTask(FileCountBySizeDao fileCountBySizeDao,
* @return Pair
*/
@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
public Pair<String, Pair<Integer, Boolean>> reprocess(OMMetadataManager omMetadataManager) {
// Map to store the count of files based on file size
Map<FileSizeCountKey, Long> fileSizeCountMap = new HashMap<>();

Expand All @@ -93,11 +93,11 @@ public Pair<String, Boolean> 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,
Expand Down Expand Up @@ -144,7 +144,8 @@ public Collection<String> getTaskTables() {
* @return Pair
*/
@Override
public Pair<String, Boolean> process(OMUpdateEventBatch events) {
public Pair<String, Pair<Integer, Boolean>> process(OMUpdateEventBatch events,
int seekPosition) {
Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
Map<FileSizeCountKey, Long> fileSizeCountMap = new HashMap<>();
final Collection<String> taskTables = getTaskTables();
Expand Down Expand Up @@ -191,7 +192,7 @@ public Pair<String, Boolean> 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.",
Expand All @@ -201,7 +202,7 @@ public Pair<String, Boolean> 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));
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand All @@ -99,27 +104,28 @@ public String getTaskName() {
}

@Override
public Pair<String, Boolean> process(OMUpdateEventBatch events) {
long startTime = System.currentTimeMillis();
boolean success = nsSummaryTaskWithFSO.processWithFSO(events);
if (!success) {
public Pair<String, Pair<Integer, Boolean>> process(
OMUpdateEventBatch events, int seekPosition) {
Pair<Integer, Boolean> 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<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
public Pair<String, Pair<Integer, Boolean>> reprocess(OMMetadataManager omMetadataManager) {
// Initialize a list of tasks to run in parallel
Collection<Callable<Boolean>> tasks = new ArrayList<>();

Expand All @@ -131,7 +137,7 @@ public Pair<String, Boolean> 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
Expand All @@ -151,15 +157,12 @@ public Pair<String, Boolean> 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();

Expand All @@ -172,7 +175,7 @@ public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
LOG.debug("Task execution time: {} milliseconds", durationInMillis);
}

return new ImmutablePair<>(getTaskName(), true);
return new ImmutablePair<>(getTaskName(), new ImmutablePair<>(0, true));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -73,15 +73,16 @@ public ReconOMMetadataManager getReconOMMetadataManager() {
protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
throws IOException {
try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
nsSummaryMap.keySet().forEach((Long key) -> {
for (Map.Entry<Long, NSSummary> 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);
throw e;
}
});
}
reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
}
}
Expand Down Expand Up @@ -210,21 +211,22 @@ protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
protected boolean flushAndCommitNSToDB(Map<Long, NSSummary> 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<Long, NSSummary> 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;
}
}*/
}
Loading
Loading