Would it make sense to have all HTables share a single thread pool, and
possibly use org.apache.hadoop.hbase.util.ShardedBlockingQueue instead
of SynchronousQueue (trunk) and instead of LinkedBlockingQueue? This
seems more in-sync with the HConnectionManager model. I understand
there's a HTablePool, and this would be a different approach to the same
problem.
I ran into this problem where our app was had > 10,000 threads. We were
creating 100 objects (used by many threads) that each have 9 HTables.
We have ~70 region servers (NrHRS), and we're using an early 0.90
client/server with some random patches. Given the code below, I
realized we were going to allocate up to 100 thread pools of 9*70
threads each. I changed the corePoolSize to 1, and later found that
this issue is already fixed in every version after the one we're using
:) and in trunk it's been switched to a SynchronousQueue.
Thanks, sorry if this has been discussed,
/charles thayer
(aka [email protected])
hbase-0.90-local::
----------------
192 int nrThreads = conf.getInt("hbase.htable.threads.max",
getCurrentNrHRS());
193 if (nrThreads == 0) {
194 nrThreads = 1; // is there a better default?
195 }
196
197 // Unfortunately Executors.newCachedThreadPool does not
allow us to
198 // set the maximum size of the pool, so we have to do it
ourselves.
199 // Must also set set corethreadpool size as with a
LinkedBlockingQueue,
200 // a new thread will not be started until the queue is full
201 this.pool = new ThreadPoolExecutor(nrThreads, nrThreads,
202 60, TimeUnit.SECONDS,
203 new LinkedBlockingQueue<Runnable>(),
204 new DaemonThreadFactory());
205 ((ThreadPoolExecutor)this.pool).allowCoreThreadTimeOut(true);
hbase-0.90.1::
-------------
182 int nrThreads = conf.getInt("hbase.htable.threads.max",
getCurrentNrHRS());
183 if (nrThreads == 0) {
184 nrThreads = 1; // is there a better default?
185 }
186
187 // Unfortunately Executors.newCachedThreadPool does not
allow us to
188 // set the maximum size of the pool, so we have to do it
ourselves.
189 this.pool = new ThreadPoolExecutor(0, nrThreads,
190 60, TimeUnit.SECONDS,
191 new LinkedBlockingQueue<Runnable>(),
192 new DaemonThreadFactory());
193 }
Current Trunk::
-------------
1024074 rawson
1094741 jdcryans int maxThreads =
conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE);
1094741 jdcryans if (maxThreads == 0) {
1094741 jdcryans maxThreads = 1; // is there a better default?
909720 stack }
909235 rawson
1094741 jdcryans // Using the "direct handoff" approach, new
threads will only be created
1094741 jdcryans // if it is necessary and will grow unbounded.
This could be bad but in HCM
1094741 jdcryans // we only create as many Runnables as there are
region servers. It means
1094741 jdcryans // it also scales when new region servers are added.
1094741 jdcryans this.pool = new ThreadPoolExecutor(1, maxThreads,
909235 rawson 60, TimeUnit.SECONDS,
1094741 jdcryans new SynchronousQueue<Runnable>(),
909235 rawson new DaemonThreadFactory());
1074331 garyh
((ThreadPoolExecutor)this.pool).allowCoreThreadTimeOut(true);
1098901 stack this.closed = false;
685009 jimk }
685009 jimk