From 6f96d09e9cb7d0f84bd6a4adec89b76b6616f105 Mon Sep 17 00:00:00 2001 From: bsglz <18031031@qq.com> Date: Thu, 6 Jul 2023 16:56:26 +0800 Subject: [PATCH] HBASE-27964 Adds a switch for compaction's delay selection feature --- .../apache/hadoop/hbase/regionserver/CompactSplit.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java index 4593a7d20020..ccb6306feeea 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java @@ -84,6 +84,8 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati public static final int DEFAULT_REGION_SERVER_REGION_SPLIT_LIMIT = 1000; public static final String HBASE_REGION_SERVER_ENABLE_COMPACTION = "hbase.regionserver.compaction.enabled"; + public static final String HBASE_REGION_SERVER_ENABLE_DELAYED_SELECTION = + "hbase.regionserver.compaction.delayedSelection.enabled"; private final HRegionServer server; private final Configuration conf; @@ -95,6 +97,7 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati private volatile Set underCompactionStores = ConcurrentHashMap.newKeySet(); private volatile boolean compactionsEnabled; + private volatile boolean delayedSelectionEnabled; /** * Splitting should not take place if the total number of regions exceed this. This is not a hard * limit to the number of regions but it is a guideline to stop splitting after number of online @@ -106,6 +109,8 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati this.server = server; this.conf = server.getConfiguration(); this.compactionsEnabled = this.conf.getBoolean(HBASE_REGION_SERVER_ENABLE_COMPACTION, true); + this.delayedSelectionEnabled = + this.conf.getBoolean(HBASE_REGION_SERVER_ENABLE_DELAYED_SELECTION, true); createCompactionExecutors(); createSplitExcecutors(); @@ -414,8 +419,8 @@ protected void requestCompactionInternal(HRegion region, HStore store, String wh } public synchronized void requestSystemCompaction(HRegion region, String why) throws IOException { - requestCompactionInternal(region, why, NO_PRIORITY, false, CompactionLifeCycleTracker.DUMMY, - DUMMY_COMPLETE_TRACKER, null); + requestCompactionInternal(region, why, NO_PRIORITY, delayedSelectionEnabled ? false : true, + CompactionLifeCycleTracker.DUMMY, DUMMY_COMPLETE_TRACKER, null); } public void requestSystemCompaction(HRegion region, HStore store, String why) throws IOException {