aswinshakil commented on code in PR #5317:
URL: https://github.com/apache/ozone/pull/5317#discussion_r1337804234
##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStore.java:
##########
@@ -150,10 +151,15 @@ public RDBStore(File dbFile, ManagedDBOptions dbOptions,
// Set CF handle in differ to be used in DB listener
rocksDBCheckpointDiffer.setSnapshotInfoTableCFHandle(
ssInfoTableCF.getHandle());
- // Finish the initialization of compaction DAG tracker by setting the
- // sequence number as current compaction log filename.
- rocksDBCheckpointDiffer.setCurrentCompactionLog(
- db.getLatestSequenceNumber());
+ // Set CF handle in differ to be store compaction log entry.
+ ColumnFamily compactionLogTableTableCF =
Review Comment:
nit: Same for below as well.
```suggestion
ColumnFamily compactionLogTableCF =
```
##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -116,16 +122,6 @@ public class RocksDBCheckpointDiffer implements
AutoCloseable,
private final String compactionLogDir;
- /**
- * Compaction log path for DB compaction history persistence.
- * This is the source of truth for in-memory SST DAG reconstruction upon
- * OM restarts.
- * <p>
- * Initialized to the latest sequence number on OM startup. The log also
rolls
- * over (gets appended to a new file) whenever an Ozone snapshot is taken.
- */
- private volatile String currentCompactionLogPath = null;
Review Comment:
Can we clean-up the other variables related to compaction or is it required
for migration to RocksDb?
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java:
##########
@@ -404,36 +403,48 @@ public void testCompactionLogBackgroundService()
cluster.getOzoneManager(leaderOM.getOMNodeId());
Assertions.assertEquals(leaderOM, newFollowerOM);
- // Prepare baseline data for compaction logs
- String currentCompactionLogPath = newLeaderOM
- .getMetadataManager()
- .getStore()
- .getRocksDBCheckpointDiffer()
- .getCurrentCompactionLogPath();
- Assertions.assertNotNull(currentCompactionLogPath);
- int lastIndex = currentCompactionLogPath.lastIndexOf(OM_KEY_PREFIX);
- String compactionLogsPath = currentCompactionLogPath
- .substring(0, lastIndex);
- File compactionLogsDir = new File(compactionLogsPath);
- Assertions.assertNotNull(compactionLogsDir);
- File[] files = compactionLogsDir.listFiles();
- Assertions.assertNotNull(files);
- int numberOfLogFiles = files.length;
- long contentLength;
- Path currentCompactionLog = Paths.get(currentCompactionLogPath);
- try (BufferedReader bufferedReader =
- Files.newBufferedReader(currentCompactionLog)) {
- contentLength = bufferedReader.lines()
- .mapToLong(String::length)
- .reduce(0L, Long::sum);
- }
+ List<CompactionLogEntry> compactionLogEntriesOnPreviousLeader =
+ getCompactionLogEntries(leaderOM);
+
+ List<CompactionLogEntry> compactionLogEntriesOnNewLeader =
+ getCompactionLogEntries(newLeaderOM);
+ Assertions.assertEquals(compactionLogEntriesOnPreviousLeader,
+ compactionLogEntriesOnNewLeader);
+
+ Assertions.assertEquals(leaderOM.getMetadataManager().getStore()
Review Comment:
Correct me if I'm wrong, Each OM has its own compaction. So the compaction
log or the sst files shouldn't be the same right?
##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -605,19 +530,40 @@ public void onCompactionCompleted(RocksDB db,
waitForTarballCreation();
- // Write input and output file names to compaction log
- appendToCurrentCompactionLog(content);
+
+ // Add the compaction log entry to Compaction log table.
+ addToCompactionLogTable(compactionLogEntry);
// Populate the DAG
// TODO: [SNAPSHOT] Once SnapshotChainManager is put into use,
// set snapshotID to snapshotChainManager.getLatestGlobalSnapshot()
- populateCompactionDAG(inputFiles, outputFiles, null,
+ populateCompactionDAG(compactionLogEntry.getInputFileInfoList(),
+ compactionLogEntry.getOutputFileInfoList(), null,
db.getLatestSequenceNumber());
}
}
};
}
+ @VisibleForTesting
+ void addToCompactionLogTable(CompactionLogEntry compactionLogEntry) {
+ // Key in the transactionId-currentTime
Review Comment:
Is there a possibility that this won't be unique and it would overwrite the
existing key? In the case of the compaction listener running at the same time
for the same trxId?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]