You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Ted Yu (JIRA)" <ji...@apache.org> on 2011/07/31 23:37:09 UTC

[jira] [Created] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
-------------------------------------------------------------------------------------------

                 Key: HBASE-4150
                 URL: https://issues.apache.org/jira/browse/HBASE-4150
             Project: HBase
          Issue Type: Bug
            Reporter: Ted Yu
             Fix For: 0.92.0


See 'Problem with hbase.client.ipc.pool.type=threadlocal in trunk' discussion started by Lars George.

>From Lars George:
Looking at HBaseClient.getConnection(...) I see this:
{code}
     synchronized (connections) {
       connection = connections.get(remoteId);
       if (connection == null) {
         connection = new Connection(remoteId);
         connections.put(remoteId, connection);
       }
     }
{code}

At the same time PoolMap.ThreadLocalPool.put is defined like this:
{code}
   public R put(R resource) {
     R previousResource = get();
     if (previousResource == null) {
...
       if (poolSize.intValue() >= maxSize) {
         return null;
       }
...
   }
{code}
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:
{code}
   * @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);
{code}
I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13078574#comment-13078574 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

Looks like we should describe the above scenario in javadoc and book.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>             Fix For: 0.92.0
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079213#comment-13079213 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

Please grant license to ASF in the future.
I will wait for a day before committing this change, wrapping the long line.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13084734#comment-13084734 ] 

Hudson commented on HBASE-4150:
-------------------------------

Integrated in HBase-TRUNK #2113 (See [https://builds.apache.org/job/HBase-TRUNK/2113/])
    HBASE-4150 update to javadoc (Karthick Sankarachary)

tedyu : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java


> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Karthick Sankarachary
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Assigned] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu reassigned HBASE-4150:
-----------------------------

    Assignee: Gary Helmling  (was: Doug Meil)

Gary agrees to obtain consensus from related parties and finish code change.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13084540#comment-13084540 ] 

stack commented on HBASE-4150:
------------------------------

+1 on doc patch.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Karthick Sankarachary
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079209#comment-13079209 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

Yep... That stops the unbounded growth of the pool. And if somebody wants a connection per thread they can set the size to Integer.MAX_VALUE.

Thanks Ted!

Is this needed:
if (maxSize != Integer.MAX_VALUE)
?
(I don't think poolSize.intValue() >= Integer.MAX_VALUE is ever true. And hence if maxSize was Integer.MAX_VALUE the 2nd if would never be reached anyway.)

I guess the documentation should state something the extend that a round-robin pool will always grow to the pool-size (at which point the connections will round-robin between threads),
and the thread-local pool will at most grow to that size (at which point further threads will fail).
Hence it makes sense to set the size of a threadlocal pool to Integer.MAX_VALUE and then control the number of threads, but the size of a round-robin pool should never be set to Integer.MAX_VALUE.


> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13083910#comment-13083910 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

+1 on the javadoc patch.
Minor comment, please put the following on one line (can be done at time of commit):
{code}
+   * @param config
+   *          configuration
{code}

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Gary Helmling (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079221#comment-13079221 ] 

Gary Helmling commented on HBASE-4150:
--------------------------------------

{quote}
The behavior now is worse than having no limit at all.
Currently, if the number of threads is greater than than the pool-size a new connection will be created for every request (see the code snippet above). That is clearly not what was intended with this.
{quote}

That certainly seems broken.

Personally I think it's a bit odd to have a limit apply to a thread local implementation, since {{ThreadLocal}} typically means I want one instance per thread.  {{ThreadLocal}} usage performs its own natural limiting, so I'd be more inclined to just remove the limit checking from {{ThreadLocalPool}}.

I also don't understand why the {{poolSizes}} variable is a static {{Map<ThreadLocalPool<?>,AtomicInteger>}}, instead of just an instance {{AtomicInteger}} variable.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Gary Helmling (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079669#comment-13079669 ] 

Gary Helmling commented on HBASE-4150:
--------------------------------------

Cool, thanks for the confirmation Karthick.  I created HBASE-4167 to continue the discussion on HTablePool. 

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Karthick Sankarachary (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079073#comment-13079073 ] 

Karthick Sankarachary commented on HBASE-4150:
----------------------------------------------

If we decide to leave things as they are, then as Ted suggested, we should describe the best practices in the javadoc and the book. If the client application comprises a small number of threads that perform a lot of scan- or batch-intensive operations, then you may want to consider using thread-local pools. For applications with a large number of threads, you may want to use a bounded round-robin pool, which will require you to fine-tune its size (a number between 5 and 10 might serve as a good starting point). At any rate, the total number of connections should not exceed the hard limits set on the underlying operating system resources. For instance, on *nix systems, the pool size should be lower than the maximum number of open file descriptors (`ulimit -n`).

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>             Fix For: 0.92.0
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Karthick Sankarachary (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079664#comment-13079664 ] 

Karthick Sankarachary commented on HBASE-4150:
----------------------------------------------

Yes, I think there may be a potential leak in {{HTablePool}}, and we could open a separate issue for that, or do it as part of this issue. 

For one thing, we might want to clear the {{tables}} variable when the {{HTablePool}} is closed (as shown below). For another, we should override {{ThreadLocalPool#get}} method so that it removes the resource, otherwise it might end up referencing a {{HTableInterface}} that's has been released.

{code}
      1 diff --git a/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java b/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
      2 index 952a3aa..c198f15 100755
      3 --- a/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
      4 +++ b/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
     13 @@ -309,6 +310,7 @@ public class HTablePool implements Closeable {
     14      for (String tableName : tables.keySet()) {
     15        closeTablePool(tableName);
     16      }
     17 +    this.tables.clear();
     18    }
{code}

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Karthick Sankarachary (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Karthick Sankarachary updated HBASE-4150:
-----------------------------------------

    Attachment: HBASE-4150-DOC.patch

Updated the javadoc in {{HBaseClient}} and {{PoolMap}} as per the comments above.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13078498#comment-13078498 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

"hbase.regionserver.handler.count" is server side config. Configuration object on client side may not have it.

I am in favor of throwing exception in the case of a thread-local pool to inform user that they should honor the limit.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>             Fix For: 0.92.0
>
>
> See 'Problem with hbase.client.ipc.pool.type=threadlocal in trunk' discussion started by Lars George.
> From Lars George:
> Looking at HBaseClient.getConnection(...) I see this:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079195#comment-13079195 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

I don't think we can leave it the way it is for the threadlocal pool.

The behavior now is worse than having no limit at all.
Currently, if the number of threads is greater than than the pool-size a new connection will be created for every *request* (see the code snippet above). That is clearly not what was intended with this.

So either threadlocal pool should have no limit (at least then we only have one connection per thread), or the it should fail when it gets past the limit.


> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu updated HBASE-4150:
--------------------------

    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:
{code}
     synchronized (connections) {
       connection = connections.get(remoteId);
       if (connection == null) {
         connection = new Connection(remoteId);
         connections.put(remoteId, connection);
       }
     }
{code}

At the same time PoolMap.ThreadLocalPool.put is defined like this:
{code}
   public R put(R resource) {
     R previousResource = get();
     if (previousResource == null) {
...
       if (poolSize.intValue() >= maxSize) {
         return null;
       }
...
   }
{code}
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:
{code}
   * @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);
{code}
I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.


  was:
See 'Problem with hbase.client.ipc.pool.type=threadlocal in trunk' discussion started by Lars George.

>From Lars George:
Looking at HBaseClient.getConnection(...) I see this:
{code}
     synchronized (connections) {
       connection = connections.get(remoteId);
       if (connection == null) {
         connection = new Connection(remoteId);
         connections.put(remoteId, connection);
       }
     }
{code}

At the same time PoolMap.ThreadLocalPool.put is defined like this:
{code}
   public R put(R resource) {
     R previousResource = get();
     if (previousResource == null) {
...
       if (poolSize.intValue() >= maxSize) {
         return null;
       }
...
   }
{code}
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:
{code}
   * @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);
{code}
I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.



> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>             Fix For: 0.92.0
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Assigned] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu reassigned HBASE-4150:
-----------------------------

    Assignee: Karthick Sankarachary  (was: Doug Meil)

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Karthick Sankarachary
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079643#comment-13079643 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

+1 to removing the limit for thread local pool.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Assigned] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu reassigned HBASE-4150:
-----------------------------

    Assignee: Doug Meil

Doug, our doc master, agrees to take over this task.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu updated HBASE-4150:
--------------------------

    Attachment: 4150.txt

Does this look good to you Lars ?

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Gary Helmling (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Gary Helmling updated HBASE-4150:
---------------------------------

    Attachment: HBASE-4150_final.patch

Committed this patch (HBASE-4150_final.patch) to trunk.

This is based on 5140-2.patch, plus a fix for TestPoolMap to reflect the non-enforcement of limit with ThreadLocalPool.

So this removes use of pool max size when using ThreadLocalPool.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079640#comment-13079640 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

bq. relative to any current trunk breakage
That's a bit hard to predict after HBASE-4158 went in - just look at recent TRUNK builds

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079718#comment-13079718 ] 

Hudson commented on HBASE-4150:
-------------------------------

Integrated in HBase-TRUNK #2084 (See [https://builds.apache.org/job/HBase-TRUNK/2084/])
    HBASE-4150  Don't enforce pool size limit with ThreadLocalPool

garyh : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/test/java/org/apache/hadoop/hbase/util/TestPoolMap.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java


> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079214#comment-13079214 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

Sorry, missed the checkbox (this is my first submission).

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13084514#comment-13084514 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

Integrate to TRUNK.

Thanks for the continued effort, Karthick.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Karthick Sankarachary
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079242#comment-13079242 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

Having an upper safety limit on the number connections (and the accompanying threads) created is not necessarily bad; that is especially true when the number of threads is somewhat hard to predict, for example when using HTablePool and HTables.

The size for the threadlocal pool should just be "reasonably large" and for the round-robin pool it should be "reasonably small".

Also, isn't PoolMap also used in HTablePool now ([HBASE-2938|https://issues.apache.org/jira/browse/HBASE-2938])? Do we want the exact same behavior there?

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Lars Hofhansl updated HBASE-4150:
---------------------------------

    Attachment: 4150-1.txt

I.e. this...

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Gary Helmling (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13078509#comment-13078509 ] 

Gary Helmling commented on HBASE-4150:
--------------------------------------

Hmm, maybe I don't understand the context, but shouldn't a thread-local pool expand to have one instance per calling thread?  I wouldn't expect any hard limit to be imposed.  Why would you want a limit in this case?  Imposing a pool size limit here is effectively just limiting number of allowed threads, but as a side effect.  Am I missing something?

It seems to me if you hit an OOME due to creating too many threads you shouldn't create so many threads.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>             Fix For: 0.92.0
>
>
> See 'Problem with hbase.client.ipc.pool.type=threadlocal in trunk' discussion started by Lars George.
> From Lars George:
> Looking at HBaseClient.getConnection(...) I see this:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Karthick Sankarachary (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13078491#comment-13078491 ] 

Karthick Sankarachary commented on HBASE-4150:
----------------------------------------------

I agree that if we don't somehow cap the pool size, we could potentially run into connection overload issues. 

How about if we enforce a hard limit on the pool size? For instance, we could use "hbase.regionserver.handler.count", for lack of a better variable, as the cap for the pool size. 

Further, in the case of a thread-local pool, we might be better off not enforcing that limit at all. If the number of threads goes over the pool size, then we could log a warning message that asks the user to switch to a round-robin pool. If that doesn't help, then we might want to consider throwing an exception, making sure to skip all of the retry logic as it bubbles up.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>             Fix For: 0.92.0
>
>
> See 'Problem with hbase.client.ipc.pool.type=threadlocal in trunk' discussion started by Lars George.
> From Lars George:
> Looking at HBaseClient.getConnection(...) I see this:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079224#comment-13079224 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

Considering what Lars said @ 04/Aug/11 04:09, it seems that ThreadLocalPool doesn't need to honor maxSize. Hence maxSize member variable can be removed.

Let's see what Karthick says.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Karthick Sankarachary (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079575#comment-13079575 ] 

Karthick Sankarachary commented on HBASE-4150:
----------------------------------------------

I tend to agree with Gary in that it doesn't really make sense to cap the thread-local pool. Having the {{PoolMap}} throw an exception seems a little too disruptive. I've attached a [patch|^4150-2.txt] that removes the limit check from {{ThreadLocalPool}} and fixes the javadoc in {{HBaseClient}}. This way, we at least ensure that the client will create as many {{Connection}} objects as threads, no more no less.

Note that in the {{PoolMap}}, we associate a resource {{Pool}} with every key, ergo the need for using a map instead of a number for the type for the {{poolSizes}} variable. So, in the case of {{HBaseClient#connections}}, we'll essentially create a {{Pool}} of {{Connection}}s for every {{ConnectionId}}, which denotes a region server. While the {{ThreadLocal}} class, which {{ThreadLocalPool}} extends, does maintain the size in {{ThreadLocalMap}}, it is unfortunately not visible to us. 

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079486#comment-13079486 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

Removing maxSize from ThreadLocalPool is a big change.
I applied the 4150.txt to TRUNK.

@Doug:
Please continue with javadoc changes.

Thanks Lars.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079552#comment-13079552 ] 

Hudson commented on HBASE-4150:
-------------------------------

Integrated in HBase-TRUNK #2082 (See [https://builds.apache.org/job/HBase-TRUNK/2082/])
    HBASE-4150 reverse patch
HBASE-4150  Potentially too many connections may be opened if ThreadLocalPool
               or RoundRobinPool is used

tedyu : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
* /hbase/trunk/CHANGES.txt

tedyu : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
* /hbase/trunk/CHANGES.txt


> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Assigned] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Gary Helmling (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Gary Helmling reassigned HBASE-4150:
------------------------------------

    Assignee: Doug Meil  (was: Gary Helmling)

Doug, over to you for any doc updates you wanted to make clarifying that pool max size configuration does not apply to ThreadLocalPool.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Doug Meil (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079348#comment-13079348 ] 

Doug Meil commented on HBASE-4150:
----------------------------------

A matter of coordination, this ticket is currently assigned to me but I'm going to wait on any doc-changes until you guys figure out what you want to do.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079212#comment-13079212 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

I guess poolSize.intValue() >= Integer.MAX_VALUE is true if you have Integer.MAX_VALUE connections (which means you also have Integer.MAX_VALUE threads, so that won't happen).

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13084518#comment-13084518 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

Thanks for the doc patch Karthick, it explains trade-offs very nicely.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Karthick Sankarachary
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu updated HBASE-4150:
--------------------------

    Attachment:     (was: 4150.txt)

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu updated HBASE-4150:
--------------------------

    Attachment: 4150.txt

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Resolved] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu resolved HBASE-4150.
---------------------------

      Resolution: Fixed
    Hadoop Flags: [Reviewed]

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Karthick Sankarachary
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150-DOC.patch, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Lars Hofhansl (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13078571#comment-13078571 ] 

Lars Hofhansl commented on HBASE-4150:
--------------------------------------

The original post was from me actually (not Lars George)...

I agree that no limit for threadlocal pool is also a good option (my #1 above).
The actual number of threads is actually a bit hard to predict because of the thread pool in HTable (and even harder when using HTablePool), though.


> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>             Fix For: 0.92.0
>
>
> See 'Problem with hbase.client.ipc.pool.type=threadlocal in trunk' discussion started by Lars George.
> From Lars George:
> Looking at HBaseClient.getConnection(...) I see this:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Gary Helmling (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079499#comment-13079499 ] 

Gary Helmling commented on HBASE-4150:
--------------------------------------

Throwing a RuntimeException where it was not thrown before in a way that could be disruptive to clients is IMO an even bigger change.  Especially in this case, this is relatively likely to occur if careful attention is not paid to aligning pool size with thread allocations.

I also don't think it's reasonable to commit a change like this when there is ongoing debate about the correct approach.  I'm willing to be convinced of alternatives, but I'm not convinced this is currently the correct approach.

-1 to this change.



> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079739#comment-13079739 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

{code}
    public ThreadLocalPool(int maxSize) {
{code}
Since maxSize is no longer used and ThreadLocalPool isn't exposed outside PoolMap, I think the ctor shouldn't take such parameter.

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt, HBASE-4150_final.patch
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Gary Helmling (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079619#comment-13079619 ] 

Gary Helmling commented on HBASE-4150:
--------------------------------------

Thanks for the updated patch Karthick.

One additional question on how this interacts with HTablePool.  In HTablePool, when obtaining a table:
{code}
  private HTableInterface findOrCreateTable(String tableName) {
    HTableInterface table = tables.get(tableName);
    if (table == null) {
      table = createHTable(tableName);
    }
    return table;
  }
{code}

In the case of {{ThreadLocalPool}}, it seems like there's an exposure here between when the table is created initially and when {{ThreadLocalPool.put()}} is called to set the thread local variable (on {{PooledHTable.close()}}).  This is independent of the issue here, so I can open a separate JIRA, just wanted to get your thoughts on if this is a potential leak.

I'll run the updated patch through tests and commit if all passes (relative to any current trunk breakage).

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Karthick Sankarachary (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Karthick Sankarachary updated HBASE-4150:
-----------------------------------------

    Attachment: 5140-2.txt

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Gary Helmling
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt, 5140-2.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4150) Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079504#comment-13079504 ] 

Ted Yu commented on HBASE-4150:
-------------------------------

Patch reversed.

@Lars:
What do you think of removing the limit checking from ThreadLocalPool ?
Effectively, maxSize would be Interger.MAX_VALUE

> Potentially too many connections may be opened if ThreadLocalPool or RoundRobinPool is used
> -------------------------------------------------------------------------------------------
>
>                 Key: HBASE-4150
>                 URL: https://issues.apache.org/jira/browse/HBASE-4150
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>            Assignee: Doug Meil
>             Fix For: 0.92.0
>
>         Attachments: 4150-1.txt, 4150.txt
>
>
> 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:
> {code}
>      synchronized (connections) {
>        connection = connections.get(remoteId);
>        if (connection == null) {
>          connection = new Connection(remoteId);
>          connections.put(remoteId, connection);
>        }
>      }
> {code}
> At the same time PoolMap.ThreadLocalPool.put is defined like this:
> {code}
>    public R put(R resource) {
>      R previousResource = get();
>      if (previousResource == null) {
> ...
>        if (poolSize.intValue() >= maxSize) {
>          return null;
>        }
> ...
>    }
> {code}
> 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:
> {code}
>    * @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);
> {code}
> I think for RoundRobinPool, we shouldn't allow maxSize to be Integer#MAX_VALUE. Otherwise connection explosion described by Lars may incur.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira