You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2013/10/23 22:13:25 UTC
svn commit: r1535146 - in /hbase/trunk:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/
hbase-common/src/main/java/org/apache/hadoop/hbase/
hbase-common/src/main/resources/
Author: ndimiduk
Date: Wed Oct 23 20:13:25 2013
New Revision: 1535146
URL: http://svn.apache.org/r1535146
Log:
HBASE-9772 Normalize new client default values
Modified:
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
hbase/trunk/hbase-common/src/main/resources/hbase-default.xml
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java?rev=1535146&r1=1535145&r2=1535146&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java Wed Oct 23 20:13:25 2013
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.client;
-
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
@@ -211,9 +210,12 @@ class AsyncProcess<CResult> {
this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
- this.maxTotalConcurrentTasks = conf.getInt("hbase.client.max.total.tasks", 100);
- this.maxConcurrentTasksPerServer = conf.getInt("hbase.client.max.perserver.tasks", 5);
- this.maxConcurrentTasksPerRegion = conf.getInt("hbase.client.max.perregion.tasks", 1);
+ this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
+ HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS);
+ this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
+ HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS);
+ this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
+ HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS);
if (this.maxTotalConcurrentTasks <= 0) {
throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1535146&r1=1535145&r2=1535146&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Wed Oct 23 20:13:25 2013
@@ -719,8 +719,9 @@ public class HConnectionManager {
long keepAliveTime = conf.getLong(
"hbase.hconnection.threads.keepalivetime", 60);
LinkedBlockingQueue<Runnable> workQueue =
- new LinkedBlockingQueue<Runnable>(128 *
- conf.getInt("hbase.client.max.total.tasks", 200));
+ new LinkedBlockingQueue<Runnable>(256 *
+ conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
+ HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
this.batchPool = new ThreadPoolExecutor(
maxThreads,
maxThreads,
Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java?rev=1535146&r1=1535145&r2=1535146&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java Wed Oct 23 20:13:25 2013
@@ -573,6 +573,38 @@ public final class HConstants {
public static long DEFAULT_HBASE_CLIENT_PAUSE = 100;
/**
+ * The maximum number of concurrent connections the client will maintain.
+ */
+ public static final String HBASE_CLIENT_MAX_TOTAL_TASKS = "hbase.client.max.total.tasks";
+
+ /**
+ * Default value of {@link #HBASE_CLIENT_MAX_TOTAL_TASKS}.
+ */
+ public static final int DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS = 100;
+
+ /**
+ * The maximum number of concurrent connections the client will maintain to a single
+ * RegionServer.
+ */
+ public static final String HBASE_CLIENT_MAX_PERSERVER_TASKS = "hbase.client.max.perserver.tasks";
+
+ /**
+ * Default value of {@link #HBASE_CLIENT_MAX_PERSERVER_TASKS}.
+ */
+ public static final int DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS = 5;
+
+ /**
+ * The maximum number of concurrent connections the client will maintain to a single
+ * Region.
+ */
+ public static final String HBASE_CLIENT_MAX_PERREGION_TASKS = "hbase.client.max.perregion.tasks";
+
+ /**
+ * Default value of {@link #HBASE_CLIENT_MAX_PERREGION_TASKS}.
+ */
+ public static final int DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS = 1;
+
+ /**
* Parameter name for server pause value, used mostly as value to wait before
* running a retry of a failed operation.
*/
Modified: hbase/trunk/hbase-common/src/main/resources/hbase-default.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/resources/hbase-default.xml?rev=1535146&r1=1535145&r2=1535146&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/resources/hbase-default.xml (original)
+++ hbase/trunk/hbase-common/src/main/resources/hbase-default.xml Wed Oct 23 20:13:25 2013
@@ -448,6 +448,26 @@ possible configurations would overwhelm
ramps up. Change this setting and hbase.client.pause to suit your workload.</description>
</property>
<property>
+ <name>hbase.client.max.total.tasks</name>
+ <value>100</value>
+ <description>The maximum number of concurrent tasks a single HTable instance will
+ send to the cluster.</description>
+ </property>
+ <property>
+ <name>hbase.client.max.perserver.tasks</name>
+ <value>5</value>
+ <description>The maximum number of concurrent tasks a single HTable instance will
+ send to a single region server.</description>
+ </property>
+ <property>
+ <name>hbase.client.max.perregion.tasks</name>
+ <value>1</value>
+ <description>The maximum number of concurrent connections the client will
+ maintain to a single Region. That is, if there is already
+ hbase.client.max.perregion.tasks writes in progress for this region, new puts
+ won't be sent to this region until some writes finishes.</description>
+ </property>
+ <property>
<name>hbase.client.scanner.caching</name>
<value>100</value>
<description>Number of rows that will be fetched when calling next