diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index b48d8fbe40fc..3591842c62d2 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -281,7 +281,7 @@ import org.apache.hadoop.ozone.om.request.OMClientRequest; import org.apache.hadoop.ozone.om.s3.S3SecretCacheProvider; import org.apache.hadoop.ozone.om.s3.S3SecretStoreProvider; -import org.apache.hadoop.ozone.om.service.CompactDBService; +import org.apache.hadoop.ozone.om.service.CompactionService; import org.apache.hadoop.ozone.om.service.DirectoryDeletingService; import org.apache.hadoop.ozone.om.service.OMRangerBGSyncService; import org.apache.hadoop.ozone.om.service.QuotaRepairTask; @@ -5381,7 +5381,16 @@ public void checkFeatureEnabled(OzoneManagerVersion feature) throws OMException public void compactOMDB(String columnFamily) throws IOException { checkAdminUserPrivilege("compact column family " + columnFamily); - new CompactDBService(this).compact(columnFamily); + // Use CompactionService if available, otherwise use static method for on-demand compaction + // The compaction happens asynchronously - we don't wait for completion + CompactionService compactionService = keyManager.getCompactionService(); + if (compactionService != null) { + compactionService.compactTableAsync(columnFamily); + } else { + // If CompactionService is not initialized (e.g., disabled), + // use static method for on-demand async compaction + CompactionService.compactTableOnDemandAsync(this, columnFamily); + } } public OMExecutionFlow getOmExecutionFlow() { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactDBService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactDBService.java deleted file mode 100644 index f39521c92190..000000000000 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactDBService.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.ozone.om.service; - -import java.io.IOException; -import java.util.concurrent.CompletableFuture; -import org.apache.hadoop.hdds.utils.db.RDBStore; -import org.apache.hadoop.hdds.utils.db.RocksDatabase; -import org.apache.hadoop.hdds.utils.db.managed.ManagedCompactRangeOptions; -import org.apache.hadoop.ozone.om.OzoneManager; -import org.apache.hadoop.util.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This service issues a compaction request for a column family of om.db. - */ -public class CompactDBService { - private static final Logger LOG = LoggerFactory.getLogger( - CompactDBService.class); - private final OzoneManager om; - - public CompactDBService(OzoneManager ozoneManager) { - this.om = ozoneManager; - } - - public CompletableFuture compact(String columnFamily) throws - IOException { - return CompletableFuture.supplyAsync(() -> { - try { - return compactAsync(columnFamily); - } catch (Exception e) { - LOG.warn("Failed to compact column family: {}", columnFamily, e); - } - return null; - }); - } - - private Void compactAsync(String columnFamilyName) throws IOException { - LOG.info("Compacting column family: {}", columnFamilyName); - long startTime = Time.monotonicNow(); - ManagedCompactRangeOptions options = - new ManagedCompactRangeOptions(); - options.setBottommostLevelCompaction( - ManagedCompactRangeOptions.BottommostLevelCompaction.kForce); - // Find CF Handler - RocksDatabase.ColumnFamily columnFamily = - ((RDBStore)om.getMetadataManager().getStore()).getDb().getColumnFamily(columnFamilyName); - ((RDBStore)om.getMetadataManager().getStore()).getDb().compactRange( - columnFamily, null, null, options); - LOG.info("Compaction of column family: {} completed in {} ms", - columnFamilyName, Time.monotonicNow() - startTime); - return null; - } -} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactionService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactionService.java index dfcc32578adf..94d4a160f048 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactionService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/CompactionService.java @@ -24,6 +24,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -134,6 +135,83 @@ private boolean shouldRun() { return !suspended.get(); } + /** + * 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 compactTableAsync(String tableName) { + return CompletableFuture.supplyAsync(() -> { + try { + compactFully(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(); + long startTime = Time.monotonicNow(); + LOG.info("Compacting column family on-demand: {}", tableName); + try (ManagedCompactRangeOptions options = new ManagedCompactRangeOptions()) { + options.setBottommostLevelCompaction(ManagedCompactRangeOptions.BottommostLevelCompaction.kForce); + options.setExclusiveManualCompaction(true); + RocksDatabase rocksDatabase = ((RDBStore) omMetadataManager.getStore()).getDb(); + + try { + // Find CF Handler + RocksDatabase.ColumnFamily columnFamily = rocksDatabase.getColumnFamily(tableName); + rocksDatabase.compactRange(columnFamily, null, null, options); + LOG.info("Compaction of column family: {} completed in {} ms", + tableName, Time.monotonicNow() - startTime); + } catch (NullPointerException ex) { + LOG.error("Unable to trigger compaction for \"{}\". Column family not found ", tableName); + throw new IOException("Column family \"" + tableName + "\" not found."); + } + } + } + + /** + * 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 compactTableOnDemandAsync( + OzoneManager ozoneManager, String tableName) { + return CompletableFuture.supplyAsync(() -> { + try { + compactTableOnDemand(ozoneManager, tableName); + return null; + } catch (Exception e) { + LOG.warn("Failed to compact column family: {}", tableName, e); + } + return null; + }); + } + protected void compactFully(String tableName) throws IOException { long startTime = Time.monotonicNow(); LOG.info("Compacting column family: {}", tableName);