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/10/11 19:44:52 UTC

svn commit: r1181970 - /hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java

Author: nspiegelberg
Date: Tue Oct 11 17:44:52 2011
New Revision: 1181970

URL: http://svn.apache.org/viewvc?rev=1181970&view=rev
Log:
Optimize the htable thread pool

Summary:
The current setting is corepool=0 and using linkedblockingqueue.
This setting is equal to singleThreadPool, only 1 thread is active at all
times.

Here is how open source trunk works:
It uses SynchronousQueue and Integer.MAX_VALUE as maximum thread number. It
will have better performance than our current setting. But the only problem is
it didn't bound the max thread size.

=======================================
 int maxThreads = conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE);
    if (maxThreads == 0) {
      maxThreads = 1; // is there a better default?
    }

    // Using the "direct handoff" approach, new threads will only be created
    // if it is necessary and will grow unbounded. This could be bad but in HCM
    // we only create as many Runnables as there are region servers. It means
    // it also scales when new region servers are added.
    this.pool = new ThreadPoolExecutor(1, maxThreads,
        60, TimeUnit.SECONDS,
        new SynchronousQueue<Runnable>(),
        new DaemonThreadFactory());
    ((ThreadPoolExecutor)this.pool).allowCoreThreadTimeOut(true);

Test Plan: will run all the unit tests and test on dev cluster

Reviewers: kannan, cgthayer, vinodv, mbautin, nspiegelberg, dhruba, jgray,
srash, pkhemani

Reviewed By: kannan

CC: dist-storage@lists, hbase@lists, dhruba, liyintang, kannan

Differential Revision: 320506

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1181970&r1=1181969&r2=1181970&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/client/HTable.java Tue Oct 11 17:44:52 2011
@@ -152,13 +152,15 @@ public class HTable implements HTableInt
       nrHRS = 10;
     }
     int nrThreads = conf.getInt("hbase.htable.threads.max", nrHRS);
-    int corePools = conf.getInt("hbase.htable.threads.corePool", 0);
     // Unfortunately Executors.newCachedThreadPool does not allow us to
     // set the maximum size of the pool, so we have to do it ourselves.
-    this.pool = new ThreadPoolExecutor(corePools, nrThreads,
+    this.pool = new ThreadPoolExecutor(nrThreads, nrThreads,
         60, TimeUnit.SECONDS,
         new LinkedBlockingQueue<Runnable>(),
         new DaemonThreadFactory());
+
+    // allow the core pool threads timeout and terminate
+    ((ThreadPoolExecutor)this.pool).allowCoreThreadTimeOut(true);
   }
 
   public Configuration getConfiguration() {