You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2012/05/05 12:37:52 UTC
svn commit: r1334388 - in
/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver:
CompactSplitThread.java Store.java
Author: mbautin
Date: Sat May 5 10:37:52 2012
New Revision: 1334388
URL: http://svn.apache.org/viewvc?rev=1334388&view=rev
Log:
[HBASE-5867] Improve Compaction Throttle Default
Summary:
We recently had a production issue where our compactions fell
behind because our compaction throttle was improperly tuned and
accidentally upgraded all compactions to the large pool. The default
from HBASE-3877 makes 1 bad assumption: the default number of flushed
files in a compaction. MinFilesToCompact should be taken into
consideration. As a default, it is less damaging for the large thread to
be slightly higher than it needs to be and only get timed-majors versus
having everything accidentally promoted.
Test Plan: - mvn test
Reviewers: JIRA, kannan, liyintang
Reviewed By: kannan
CC: stack
Differential Revision: https://reviews.facebook.net/D2943
Modified:
hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=1334388&r1=1334387&r2=1334388&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Sat May 5 10:37:52 2012
@@ -44,7 +44,6 @@ public class CompactSplitThread {
private final ThreadPoolExecutor largeCompactions;
private final ThreadPoolExecutor smallCompactions;
private final ThreadPoolExecutor splits;
- private final long throttleSize;
/* The default priority for user-specified compaction requests.
* The user gets top priority unless we have blocking compactions (Pri <= 0)
@@ -62,26 +61,20 @@ public class CompactSplitThread {
int largeThreads = Math.max(1, conf.getInt(
"hbase.regionserver.thread.compaction.large", 1));
int smallThreads = conf.getInt(
- "hbase.regionserver.thread.compaction.small", 0);
- throttleSize = conf.getLong(
- "hbase.regionserver.thread.compaction.throttle", 0);
+ "hbase.regionserver.thread.compaction.small", 1);
int splitThreads = conf.getInt("hbase.regionserver.thread.split", 1);
- // if we have throttle threads, make sure the user also specified size
- Preconditions.checkArgument(smallThreads == 0 || throttleSize > 0);
+ Preconditions.checkArgument(largeThreads > 0 && smallThreads > 0);
this.largeCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
this.largeCompactions
.setRejectedExecutionHandler(new CompactionRequest.Rejection());
- if (smallThreads <= 0) {
- this.smallCompactions = null;
- } else {
- this.smallCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
- 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
- this.smallCompactions
- .setRejectedExecutionHandler(new CompactionRequest.Rejection());
- }
+ this.smallCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
+ 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
+ this.smallCompactions
+ .setRejectedExecutionHandler(new CompactionRequest.Rejection());
+
this.splits = (ThreadPoolExecutor) Executors
.newFixedThreadPool(splitThreads);
}
@@ -161,18 +154,13 @@ public class CompactSplitThread {
if (priority != NO_PRIORITY) {
cr.setPriority(priority);
}
- ThreadPoolExecutor pool = largeCompactions;
- if (smallCompactions != null && throttleSize > cr.getSize()) {
- // smallCompactions is like the 10 items or less line at Walmart
- pool = smallCompactions;
- }
+ // smallCompactions: like the 10 items or less line at Walmart
+ ThreadPoolExecutor pool = s.throttleCompaction(cr.getSize())
+ ? largeCompactions : smallCompactions;
pool.execute(cr);
if (LOG.isDebugEnabled()) {
- String type = "";
- if (smallCompactions != null) {
- type = (pool == smallCompactions) ? "Small " : "Large ";
- }
- LOG.debug(type + "Compaction requested: " + cr
+ LOG.debug((pool == smallCompactions) ? "Small " : "Large "
+ + "Compaction requested: " + cr
+ (why != null && !why.isEmpty() ? "; Because: " + why : "")
+ "; " + this);
}
Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1334388&r1=1334387&r2=1334388&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Sat May 5 10:37:52 2012
@@ -1919,6 +1919,13 @@ public class Store extends SchemaConfigu
return this.blockingStoreFileCount - this.storefiles.size();
}
+ boolean throttleCompaction(long compactionSize) {
+ long throttlePoint = conf.getLong(
+ "hbase.regionserver.thread.compaction.throttle",
+ 2 * this.minFilesToCompact * this.region.memstoreFlushSize);
+ return compactionSize > throttlePoint;
+ }
+
HRegion getHRegion() {
return this.region;
}