You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/05/11 23:50:37 UTC
svn commit: r1102105 - in /hbase/trunk: CHANGES.txt
src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
Author: nspiegelberg
Date: Wed May 11 21:50:37 2011
New Revision: 1102105
URL: http://svn.apache.org/viewvc?rev=1102105&view=rev
Log:
HBASE-3877 Determine Proper Defaults for Compaction ThreadPools
Modified:
hbase/trunk/CHANGES.txt
hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1102105&r1=1102104&r2=1102105&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed May 11 21:50:37 2011
@@ -214,6 +214,7 @@ Release 0.91.0 - Unreleased
(dhruba borthakur)
HBASE-3797 StoreFile Level Compaction Locking
HBASE-1476 Multithreaded Compactions
+ HBASE-3877 Determine Proper Defaults for Compaction ThreadPools
TASKS
HBASE-3559 Move report of split to master OFF the heartbeat channel
Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=1102105&r1=1102104&r2=1102105&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Wed May 11 21:50:37 2011
@@ -28,6 +28,8 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import com.google.common.base.Preconditions;
@@ -70,9 +72,19 @@ public class CompactSplitThread implemen
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);
+ if (conf.get("hbase.regionserver.thread.compaction.throttle") != null) {
+ throttleSize = conf.getLong(
+ "hbase.regionserver.thread.compaction.throttle", 0);
+ } else {
+ // we have a complicated default. see HBASE-3877
+ long flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
+ HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
+ long splitSize = conf.getLong("hbase.hregion.max.filesize",
+ HConstants.DEFAULT_MAX_FILE_SIZE);
+ throttleSize = Math.min(flushSize * 2, splitSize / 2);
+ }
+
int splitThreads = conf.getInt("hbase.regionserver.thread.split", 1);
// if we have throttle threads, make sure the user also specified size