hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mbau...@apache.org
Subject svn commit: r1293194 - /hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HTable.java
Date Fri, 24 Feb 2012 11:38:04 GMT
Author: mbautin
Date: Fri Feb 24 11:38:03 2012
New Revision: 1293194

URL: http://svn.apache.org/viewvc?rev=1293194&view=rev
Log:
[HBASE-3767] Using direct handoff (SynchronousQueue) for HTable's thread pool

Summary:
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.

Tag this master since the open source has fixed this problem.

Test Plan: Running all the unit tests and I will using the LoadTest to verify
the number thread created.

Reviewers: kannan

Reviewed By: kannan

CC: hbase-eng@lists, dhruba, pkhemani

Differential Revision: https://phabricator.fb.com/D414190

Task ID: 938765

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

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1293194&r1=1293193&r2=1293194&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/client/HTable.java Fri Feb
24 11:38:03 2012
@@ -19,6 +19,24 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -37,25 +55,6 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Writables;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-
 /**
  * Used to communicate with a single HBase table.
  *
@@ -151,20 +150,16 @@ public class HTable implements HTableInt
       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
     this.maxKeyValueSize = conf.getInt("hbase.client.keyvalue.maxsize", -1);
 
-    int nrHRS = getCurrentNrHRS();
-    if (nrHRS == 0) {
-      // No servers running -- set default of 10 threads.
-      nrHRS = 10;
-    }
-    int nrThreads = conf.getInt("hbase.htable.threads.max", nrHRS);
-    // 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(nrThreads, nrThreads,
+    int maxThreads = conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE);
+
+    // 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 LinkedBlockingQueue<Runnable>(),
+        new SynchronousQueue<Runnable>(),
         new DaemonThreadFactory());
-
-    // allow the core pool threads timeout and terminate
     ((ThreadPoolExecutor)this.pool).allowCoreThreadTimeOut(true);
   }
 



Mime
View raw message