Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-27964 Adds a switch for compaction's delay selection feature #5318

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -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;
Expand All @@ -95,6 +97,7 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
private volatile Set<String> 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
Expand All @@ -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();

Expand Down Expand Up @@ -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 {
Expand Down