Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
None
-
None
-
None
-
Reviewed
Description
See 'Problem with hbase.client.ipc.pool.type=threadlocal in trunk' discussion started by Lars George.
From Lars Hofhansl:
Looking at HBaseClient.getConnection(...) I see this:
synchronized (connections) { connection = connections.get(remoteId); if (connection == null) { connection = new Connection(remoteId); connections.put(remoteId, connection); } }
At the same time PoolMap.ThreadLocalPool.put is defined like this:
public R put(R resource) { R previousResource = get(); if (previousResource == null) { ... if (poolSize.intValue() >= maxSize) { return null; } ... }
So... If the ThreadLocalPool reaches its capacity it always returns null and hence all new threads will create a
new connection every time getConnection is called!
I have also verified with a test program that works fine as long as the number of client threads (which include
the threads in HTable's threadpool of course) is < poolsize. Once that is no longer the case the number of
connections "explodes" and the program dies with OOMEs (mostly because each Connection is associated with
yet another thread).
It's not clear what should happen, though. Maybe (1) the ThreadLocalPool should not have a limit, or maybe
(2) allocations past the pool size should throw an exception (i.e. there's a hard limit), or maybe (3) in that case
a single connection is returned for all threads while the pool it over its limit or (4) we start round robin with the other
connection in the other thread locals.
For #1 means that the number of client threads needs to be more carefully managed by the client app.
In this case it would also be somewhat pointless that Connection have their own threads, we just pass stuff
between threads.
#2 would work, but puts more logic in the client.
#3 would lead to hard to debug performance issues.
And #4 is messy
From Ted Yu:
For HBaseClient, at least the javadoc doesn't match:
* @param config configuration * @return either a {@link PoolType#Reusable} or {@link PoolType#ThreadLocal} */ private static PoolType getPoolType(Configuration config) { return PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE), PoolType.RoundRobin, PoolType.ThreadLocal);
I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.