Can you tell when these WARN messages are produced? Is it related to the
creation of the connection object or one of the HTable instances?

On Thu, Feb 26, 2015 at 7:27 AM, Marcelo Valle (BLOOMBERG/ LONDON) <
mvallemil...@bloomberg.net> wrote:

> Nick,
>
> I tried what you suggested, 1 HConnection and 1 Configuration for the
> entire app:
>
> this.config = HBaseConfiguration.create();
> this.connection = HConnectionManager.createConnection(config);
>
> And Threaded pooled HTableInterfaces:
>
> final HConnection lconnection = this.connection;
> this.tlTable = new ThreadLocal<HTableInterface>() {
> @Override
> protected HTableInterface initialValue() {
> try {
> return lconnection.getTable("HBaseSerialWritesPOC");
> // return new HTable(tlConfig.get(),
> // "HBaseSerialWritesPOC");
> } catch (IOException e) {
> throw new RuntimeException(e);
> }
> }
> };
>
> I started getting this error in my application:
>
> 2015-02-26 10:23:17,833 INFO [main-SendThread(xxx)] zookeeper.ClientCnxn
> (ClientCnxn.java:logStartConnect(966)) - Opening socket connection to
> server xxx. Will not attempt to authenticate using SASL (unknown error)
> 2015-02-26 10:23:17,834 INFO [main-SendThread(xxx)] zookeeper.ClientCnxn
> (ClientCnxn.java:primeConnection(849)) - Socket connection established to
> xxx, initiating session
> 2015-02-26 10:23:17,836 WARN [main-SendThread(xxx)] zookeeper.ClientCnxn
> (ClientCnxn.java:run(1089)) - Session 0x0 for server xxx, unexpected error,
> closing socket connection and attempting reconnect
> java.io.IOException: Connection reset by peer
> at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:192)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:379)
> at
> org.apache.zookeeper.ClientCnxnSocketNIO.doIO(ClientCnxnSocketNIO.java:68)
> at
> org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:355)
> at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1068)
>
>
> -Marcelo
>
> From: ndimi...@gmail.com
> Subject: Re: HBase connection pool
>
> Okay, looks like you're using a implicitly managed connection. It should
> be fine to share a single config instance across all threads. The advantage
> of HTablePool over this approach is that the number of HTables would be
> managed independently from the number of Threads. This may or not be a
> concern for you, based on your memory requirements, &c. In your case,
> you're not specifying an ExecutorService per HTable, so the HTable
> instances will be relatively light weight. Each table will manage it's own
> write buffer, which can be shared by multiple threads when autoFlush is
> disabled and HTablePool is used. This may or may not be desirable,
> depending on your use-case.
>
> For what it's worth, HTablePool is marked deprecated in 1.0, will likely
> be removed in 2.0. To "future proof" this code, I would move to a single
> shared HConnection for the whole application, and a thread-local HTable
> created from/with that connection.
>
> -n
>
> On Wed, Feb 25, 2015 at 10:53 AM, Marcelo Valle (BLOOMBERG/ LONDON) <
> mvallemil...@bloomberg.net> wrote:
>
>> Hi Nick,
>>
>> I am using HBase version 0.96, I sent the link from version 0.94 because
>> I haven't found the java API docs for 0.96, sorry about that.
>> I have created the HTable directly from the config object, as follows:
>>
>> this.tlConfig = new ThreadLocal<Configuration>() {
>>
>> @Override
>> protected Configuration initialValue() {
>> return HBaseConfiguration.create();
>> }
>> };
>> this.tlTable = new ThreadLocal<HTable>() {
>> @Override
>> protected HTable initialValue() {
>> try {
>> return new HTable(tlConfig.get(), "HBaseSerialWritesPOC");
>> } catch (IOException e) {
>> throw new RuntimeException(e);
>> }
>> }
>> };
>>
>> I am now sure if the Configuration object should be 1 per thread as well,
>> maybe I could share this one?
>>
>> So, just to clarify, would I get any advantage using HTablePool object
>> instead of ThreadLocal<HTable> as I did?
>>
>> -Marcelo
>>
>> From: ndimi...@gmail.com
>> Subject: Re: HBase connection pool
>>
>> Hi Marcelo,
>>
>> First thing, to be clear, you're working with a 0.94 release? The reason
>> I ask is we've been doing some work in this area to improve things, so
>> semantics may be slightly different between 0.94, 0.98, and 1.0.
>>
>> How are you managing the HConnection object (or are you)? How are you
>> creating your HTable instances? These will determine how the connection is
>> obtained and used in relation to HTables.
>>
>> In general, multiple HTable instances connected to tables in the same
>> cluster should be sharing the same HConnection instance. This is handled
>> explicitly when you manage your own HConnection and HTables (i.e.,
>> HConnection conn = ... ; HTable t = new HTable(TABLE_NAME, conn); ) It's
>> handled implicitly when you construct via Configuration objects (HTable t =
>> new HTable(conf, TABLE_NAME); ) This implicit option is going away in
>> future versions.
>>
>> HTable is not safe for concurrent access because of how the write path is
>> implemented (at least; there may be other portions that I'm not as familiar
>> with). You should be perfectly fine to have an HTable per thread in a
>> ThreadLocal.
>>
>> -n
>>
>> On Wed, Feb 25, 2015 at 9:41 AM, Marcelo Valle (BLOOMBERG/ LONDON) <
>> mvallemil...@bloomberg.net> wrote:
>>
>>> In HBase API, does 1 HTable object means 1 connection to each region
>>> server (just for 1 table)?
>>>
>>> The docs say (
>>> http://hbase.apache.org/0.94/apidocs/org/apache/hadoop/hbase/client/HTable.html
>>> ):
>>> "This class is not thread safe for reads nor write."
>>>
>>> I got confused, as I saw there is a HTablePool class, but it's only for
>>> a table as well, can't connections be reused for more than 1 table?
>>>
>>> In my java application, I used ThreadLocal variables
>>> (ThreadLocal<HTable>) to create an HTable variable per thread. If I do
>>> several operations on each thread, I should still use the same connection,
>>> right?
>>>
>>> []s
>>
>>
>>
>>
>
>

Reply via email to