hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Henning Blohm <henning.bl...@zfabrik.de>
Subject Re: limiting user threads on client
Date Tue, 14 Mar 2017 07:51:20 GMT
I am using a single Connection (HBase 1.0.0) created using 
ConnectionFactory.createConnection(config) that is kept alive virtually 
forever and actual client requests call connection.getTable(...), do 
their work, and finally call table.close() when done.

Setting hbase.hconnection.threads.max and hbase.hconnection.threads.core 
(in particular the latter) provides the desired result. A (what I 
consider) heavy write operation (10 threads in parallel) now creates 
only 20 hconnection threads rather than 256.

Thanks a lot!
Henning


On 03/14/2017 12:26 AM, Enis Söztutar wrote:
> There are different thread pools in the client, and some of the thread
> pools depend on how are you constructing connection and table instances.
>
> The first thread pool is the one owned by the connection. If you are using
> ConnectionFactory.createConnection() (which you should) then this is the
> property that controls how many of the threads in the connection:
>
> hbase.hconnection.threads.max
>
> This one configures when the threads will be discarded:
>
> hbase.hconnection.threads.keepalivetime
>
> You can also give your own thread pool to the Connection object if you want
> to control threading behavior.
> If you are creating HTable or Table objects from Connection, then by
> default they share the same thread pool, so you do not have to do anything.
> Otherwise, the HTable objects can have their own thread pools as well.
>
> Then, there are RPC-level thread pools. In 1.x versions (unless you have
> netty based async RPC), there is one thread per regionserver that the
> client talks to. I don't think there is a limit of how many of these the
> client can have at a single time. So, if the client ends up doing RPCs to
> many servers, there will be one thread per server.
>
> You should use jstack or kill -3 to inspect the hbase client threads
> probably.
>
> Enis
> On Mon, Mar 13, 2017 at 2:57 PM, anil gupta <anilgupta84@gmail.com> wrote:
>
>> I think you need to set that property before you make HBaseConfiguration
>> object. Have you tried that?
>>
>> On Mon, Mar 13, 2017 at 10:24 AM, Henning Blohm <henning.blohm@zfabrik.de>
>> wrote:
>>
>>> Unfortunately it doesn't seem to make a difference.
>>>
>>> I see that the configuration has hbase.htable.threads.max=1 right before
>>> setting up the Connection but then I still get hundreds of
>>>
>>> hconnection-***
>>>
>>> threads. Is that actually Zookeeper?
>>>
>>> Thanks,
>>> Henning
>>>
>>> On 13.03.2017 17:28, Ted Yu wrote:
>>>
>>>> Are you using Java client ?
>>>> See the following in HTable :
>>>>
>>>>     public static ThreadPoolExecutor getDefaultExecutor(Configuration
>>>> conf) {
>>>>
>>>>       int maxThreads = conf.getInt("hbase.htable.threads.max", Integer.
>>>> MAX_VALUE);
>>>>
>>>> FYI
>>>>
>>>> On Mon, Mar 13, 2017 at 9:14 AM, Henning Blohm <
>> henning.blohm@zfabrik.de>
>>>> wrote:
>>>>
>>>> Hi,
>>>>> I am running an HBase client on a very resource limited machine. In
>>>>> particular numproc is limited so that I frequently get "Cannot create
>>>>> native thread" OOMs. I noticed that, in particular in write situations,
>>>>> the
>>>>> hconnection pool grows into the hundreds of threads - even when at most
>>>>> writing with less than ten application threads. Threads are discarded
>>>>> again
>>>>> after some minutes.
>>>>>
>>>>> In conjunction with other programs running on that machine, this
>>>>> sometimes
>>>>> leads to an "overload" situation.
>>>>>
>>>>> Is there a way to keep thread pool usage limited - or in some closer
>>>>> relation with the actual concurrency required?
>>>>>
>>>>> Thanks,
>>>>>
>>>>> Henning
>>>>>
>>>>>
>>>>>
>>>>>
>>
>> --
>> Thanks & Regards,
>> Anil Gupta
>>


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