sadanand48 commented on code in PR #9663:
URL: https://github.com/apache/ozone/pull/9663#discussion_r2772534849
##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactionService.java:
##########
@@ -134,25 +131,68 @@ private boolean shouldRun() {
return !suspended.get();
}
- protected void compactFully(String tableName) throws IOException {
- long startTime = Time.monotonicNow();
- LOG.info("Compacting column family: {}", tableName);
- try (ManagedCompactRangeOptions options = new
ManagedCompactRangeOptions()) {
-
options.setBottommostLevelCompaction(ManagedCompactRangeOptions.BottommostLevelCompaction.kForce);
- options.setExclusiveManualCompaction(true);
- RocksDatabase rocksDatabase = ((RDBStore)
omMetadataManager.getStore()).getDb();
+ /**
+ * Compact a specific table asynchronously. This method returns immediately
+ * with a CompletableFuture that completes when the compaction finishes.
+ * This is useful for on-demand compaction requests (e.g., via admin RPC)
+ * where the caller doesn't need to wait for completion.
+ *
+ * @param tableName the name of the table to compact
+ * @return CompletableFuture that completes when compaction finishes
+ */
+ public CompletableFuture<Void> compactTableAsync(String tableName) {
+ return CompletableFuture.supplyAsync(() -> {
+ try {
+ CompactDBUtil.compactTable(omMetadataManager, tableName);
+ return null;
+ } catch (Exception e) {
+ LOG.warn("Failed to compact column family: {}", tableName, e);
+ }
+ return null;
+ });
+ }
+ /**
+ * Compact a specific table on-demand without requiring the table
+ * to be in the configured compaction list. This is useful for
+ * ad-hoc compaction requests (e.g., via admin RPC).
+ *
+ * @param ozoneManager the OzoneManager instance
+ * @param tableName the name of the table to compact
+ * @throws IOException if compaction fails or table is not found
+ */
+ public static void compactTableOnDemand(OzoneManager ozoneManager, String
tableName)
+ throws IOException {
+ OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+ CompactDBUtil.compactTable(omMetadataManager, tableName);
+ }
+
+ /**
+ * Compact a specific table on-demand asynchronously without requiring the
table
+ * to be in the configured compaction list. This method returns immediately
+ * with a CompletableFuture that completes when the compaction finishes.
+ * This is useful for ad-hoc compaction requests (e.g., via admin RPC)
+ * where the caller doesn't need to wait for completion.
+ *
+ * @param ozoneManager the OzoneManager instance
+ * @param tableName the name of the table to compact
+ * @return CompletableFuture that completes when compaction finishes
+ */
+ public static CompletableFuture<Void> compactTableOnDemandAsync(
Review Comment:
Lets remove the static methods here and simply keep these two methods:
```java
public CompletableFuture<Void> compactTableAsync(String tableName) {
return CompactDBUtil.compactTableAsync(omMetadataManager, tableName);
}
protected void compactFully(String tableName) throws IOException {
CompactDBUtil.compactTable(omMetadataManager, tableName);
}
```
add the async method in CompactDBUtil:
``` java
public static CompletableFuture<Void> compactTableAsync(OMMetadataManager
metadataManager, String tableName) {
return CompletableFuture.runAsync(() -> {
try {
compactTable(metadataManager, tableName);
} catch (Exception e) {
LOG.warn("Failed to compact column family: {}", tableName, e);
}
});
}
```
and directly call this in OzoneManager.java without the if else branching
--
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]