Skip to content
Merged
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 @@ -501,7 +501,9 @@ ImmutablePair<Boolean, Long> innerGetAndApplyDeltaUpdatesFromOM(long fromSequenc
}
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 @@ -34,8 +34,6 @@
import java.util.List;
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.hdds.utils.db.Table;
Expand Down Expand Up @@ -82,7 +80,7 @@ public ContainerKeyMapperTask(ReconContainerMetadataManager
* (container, key) -&gt; count to Recon Container DB.
*/
@Override
public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
public TaskResult reprocess(OMMetadataManager omMetadataManager) {
long omKeyCount = 0;

// In-memory maps for fast look up and batch write
Expand Down Expand Up @@ -118,7 +116,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 buildTaskResult(false);
}
omKeyCount++;
}
Expand All @@ -131,7 +129,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 buildTaskResult(false);
}

LOG.debug("Completed 'reprocess' of ContainerKeyMapperTask.");
Expand All @@ -142,9 +140,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 buildTaskResult(false);
}
return new ImmutablePair<>(getTaskName(), true);
return buildTaskResult(true);
}

private boolean flushAndCommitContainerKeyInfoToDB(
Expand Down Expand Up @@ -189,7 +187,8 @@ public Collection<String> getTaskTables() {
}

@Override
public Pair<String, Boolean> process(OMUpdateEventBatch events) {
public TaskResult process(OMUpdateEventBatch events,
Map<String, Integer> subTaskSeekPosMap) {
Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
int eventCount = 0;
final Collection<String> taskTables = getTaskTables();
Expand Down Expand Up @@ -246,18 +245,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 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 ImmutablePair<>(getTaskName(), false);
return buildTaskResult(false);
}
LOG.debug("{} successfully processed {} OM DB update event(s) in {} milliseconds.",
getTaskName(), eventCount, (System.currentTimeMillis() - startTime));
return new ImmutablePair<>(getTaskName(), true);
return buildTaskResult(true);
}

private void writeToTheDB(Map<ContainerKeyPrefix, Integer> containerKeyMap,
Expand Down
Loading