Home | About | Sematext search-lucene.com search-hadoop.com
 Search Hadoop and all its subprojects:

Switch to Threaded View
HBase >> mail # user >> Problem with hbase.client.ipc.pool.type=threadlocal in trunk


Copy link to this message
-
Re: Problem with hbase.client.ipc.pool.type=threadlocal in trunk
If you mean Karthick Sankarachary, he does not hang around mailing list much
. Let me go and get him.

On Tue, Aug 2, 2011 at 3:06 PM, Ted Yu <[EMAIL PROTECTED]> wrote:

> I created https://issues.apache.org/jira/browse/HBASE-4150
>
> On Tue, Aug 2, 2011 at 3:02 PM, lars hofhansl <[EMAIL PROTECTED]> wrote:
>
> > Yep, the RR pool has a similar issue. Maybe the maximum number of
> > connection should be related to the number of cores on the client.
> > Something like 2 x #cores?
> >
> > For the threadLocal case that I mentioned below it would be hard to find
> a
> > useful hard limit, since it is hard to foresee the number of threads that
> > will be accessing a region (because of the threadpool in HTable).
> >
> > More generally: Should HTable have a threadpool at all? Or should the
> > pooling be happening on the Connection level?
> >
> > Lastly should I create an issue? :)
> >
> > ------------------------------
> > *From:* Ted Yu <[EMAIL PROTECTED]>
> > *To:* [EMAIL PROTECTED]
> > *Cc:* lars hofhansl <[EMAIL PROTECTED]>
> > *Sent:* Friday, July 29, 2011 9:52 AM
> >
> > *Subject:* Re: Problem with hbase.client.ipc.pool.type=threadlocal in
> > trunk
> >
> > I am waiting for Karthick to shed some light on this.
> >
> > On Fri, Jul 29, 2011 at 9:47 AM, Stack <[EMAIL PROTECTED]> wrote:
> >
> > Sounds good.  Issue?
> > St.Ack
> >
> >
> > On Thu, Jul 28, 2011 at 10:46 PM, Ted Yu <[EMAIL PROTECTED]> wrote:
> > > 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 <[EMAIL PROTECTED]>
> > 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 <[EMAIL PROTECTED]>
> > >> To: [EMAIL PROTECTED]
> > >> 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 <[EMAIL PROTECTED]> 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;
> > >> >        }
> > >> > ...