hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ted Yu <yuzhih...@gmail.com>
Subject Re: Problem with hbase.client.ipc.pool.type=threadlocal in trunk
Date Fri, 29 Jul 2011 05:46:02 GMT
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.

Cheers

On Thu, Jul 28, 2011 at 10:27 PM, lars hofhansl <lhofhansl@yahoo.com> wrote:

> Yeah I looked at that one too, but I assumed that the rationale was to keep
> creating connectionsuntil the pool is full, which makes
> some sense for a round robin pool. Otherwise it is not clear to me how the
> pool grow at all?
>
>
> (And at least it can be usefully controlled with the pool.size parameter.)
>
>
> Cheers indeed
>
>
> ----- Original Message -----
> From: Ted Yu <yuzhihong@gmail.com>
> To: user@hbase.apache.org
> Cc:
> Sent: Thursday, July 28, 2011 10:07 PM
> Subject: Re: Problem with hbase.client.ipc.pool.type=threadlocal in trunk
>
> Looking at PoolMap, I have some question as well.
> For RoundRobinPool, I don't understand the following method:
>     public R get() {
>       if (size() < maxSize) {
>         return null;
>       }
> Should the above condition be?
>       if (size() <= 0) {
>
> For the issue Lars raised, I prefer solution number 2.
>
> Cheers
>
> On Thu, Jul 28, 2011 at 9:12 PM, lars <lhofhansl@yahoo.com> wrote:
>
> > 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 :)
> >
> > Any other options?
> >
> > Maybe for now just do #2 (or #1)?
> >
> > -- Lars
> >
> >
>
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message