I did more research and found the issue.

The TableInputFormat creates an HTable using a new Configuration object, and it 
never cleans it up. When running a Mapper, the TableInputFormat is instantiated 
and the ZK connection is created. While this connection is not explicitly 
cleaned up, the Mapper process eventually exits and thus the connection is 
closed. Ideally the TableRecordReader would close the connection in its close() 
method rather than relying on the process to die for connection cleanup. This 
is fairly easy to implement by overriding TableRecordReader, and also 
overriding TableInputFormat to specify the new record reader.

The leak occurs when the JobClient is initializing and needs to retrieves the 
splits. To get the splits, it instantiates a TableInputFormat. Doing so creates 
a ZK connection that is never cleaned up. Unlike the mapper, however, my job 
client process does not die. Thus the ZK connections accumulate.

I was able to fix the problem by writing my own TableInputFormat that does not 
initialize the HTable in the getConf() method and does not have an HTable 
member variable. Rather, it has a variable for the table name. The HTable is 
instantiated where needed and then cleaned up. For example, in the getSplits() 
method, I create the HTable, then close the connection once the splits are 
retrieved. I also create the HTable when creating the record reader, and I have 
a record reader that closes the connection when done.

Calling HConnectionManager.deleteAllConnections() is not desirable in my case, 
as I may have some connections that I do not want deleted.


On Apr 16, 2011, at 3:56 AM, Ted Yu wrote:

> I think you should call this method of HTablePool:
>  public void closeTablePool(final String tableName) throws IOException {
> 
> Actually you only use HTablePool in populateTable(), HTable should be enough
> for you.
> 
> I have logged https://issues.apache.org/jira/browse/HBASE-3791 for ease of
> debugging.
> 
> I think if you place this call:
> HConnectionManager.deleteAllConnections(true);
> on line 52 before calling obj.wait(), situation should be different.
> 
> Cheers
> 
> On Fri, Apr 15, 2011 at 11:56 PM, Bryan Keller <[email protected]> wrote:
> 
>> FWIW, I created a test program that demonstrates the issue. The program
>> creates an HBase table, populates it with 10 rows, then runs a simple
>> map-reduce job 10 times in succession, and then goes into a wait state. The
>> test uses gradle so you'll need to download that.
>> 
>> Before running, telnet to Zookeeper and type 'stats' to get the
>> connections. Then run the program using 'gradle run'. Finally, telnet to
>> Zookeeper again and type 'stats' to get the connections.
>> 
>> I'd be interested to see if others are seeing the same behavior I am.
>> 
>> You can download the code here:
>> http://www.vancameron.net/HBaseMR.zip
>> 
>> I'll open a JIRA issue after I do a little more research into the problem.
>> 
>> On Apr 15, 2011, at 4:19 PM, Ted Yu wrote:
>> 
>>> Bryan:
>>> Thanks for reporting this issue.
>>> TableOutputFormat.TableRecordWriter calls the following in close():
>>>     HConnectionManager.deleteAllConnections(true);
>>> But there is no such call in TableInputFormat / TableInputFormatBase /
>>> TableRecordReader
>>> 
>>> Do you mind filing a JIRA ?
>>> 
>>> On Fri, Apr 15, 2011 at 3:41 PM, Bryan Keller <[email protected]> wrote:
>>> 
>>>> I am having this same problem. After every run of my map-reduce job
>> which
>>>> uses TableInputFormat, I am leaking one ZK connection. The connections
>> that
>>>> are not being cleaned up are connected to the node that submitted the
>> job,
>>>> not the cluster nodes.
>>>> 
>>>> I tried explicitly cleaning up the connection using
>>>> HConnectionManager.deleteConnection(config, true) after the job runs,
>> but
>>>> this has no effect. ZK still retains one connection per job run and
>> never
>>>> releases it. Eventually I run out of ZK connections even if I set
>> maxCnxns
>>>> very high (e.g. 600).
>>>> 
>>>> This happened for me with CDH3B4 and is still happening with the CDH3
>>>> release.
>>>> 
>>>> 
>>>> 
>>>> On Mar 23, 2011, at 3:27 PM, Todd Lipcon wrote:
>>>> 
>>>>> Hi Dmitriy,
>>>>> 
>>>>> Are you submitting these MR jobs on a cluster? Which machines are
>>>>> leaking the connections? Is it the cluster nodes or the node where you
>>>>> submitted the job?
>>>>> 
>>>>> After a job is complete, the JVMs that ran the tasks should be
>>>>> completely torn down and thus should not be able to hang onto a
>>>>> connection.
>>>>> 
>>>>> -Todd
>>>>> 
>>>>> On Wed, Mar 23, 2011 at 2:24 PM, Dmitriy Lyubimov <[email protected]>
>>>> wrote:
>>>>>> yes i am passing destroyProxy=true. But according to the code, it
>>>>>> doesn't affect closing zookeeper connection (it should be closed
>>>>>> anyway) but i  have +1 zk connection each time i run the MR job still.
>>>>>> 
>>>>>> -d
>>>>>> 
>>>>>> On Wed, Mar 23, 2011 at 2:22 PM, Ted Yu <[email protected]> wrote:
>>>>>>> I assume you passed true as second parameter to deleteConnection().
>>>>>>> 
>>>>>>> On Wed, Mar 23, 2011 at 1:54 PM, Dmitriy Lyubimov <[email protected]
>>> 
>>>> wrote:
>>>>>>> 
>>>>>>>> Hi,
>>>>>>>> 
>>>>>>>> I am experiencing severe connection leak in my MR client that uses
>>>>>>>> Hbase as input/output . Every job that uses TableInputFormat leaks 1
>>>>>>>> zookeeper connection per run as evidenced by netstat.
>>>>>>>> 
>>>>>>>> I understand that the way HTable manages connections now is it
>> creates
>>>>>>>> a new HBase (and also Zookeeper) connection per each instance of
>>>>>>>> Configuration it is initialized with. By looking at the code of the
>>>>>>>> TableInputFormat class, i see that it creates HTable in the front
>> end
>>>>>>>> during configuration (of course, it probably needs to use it to
>>>>>>>> determine region splits).
>>>>>>>> 
>>>>>>>> Since i have to configure each job individually, i must create a new
>>>>>>>> instance of Configuration. Thus, i am not able to use shared HBase
>>>>>>>> connections (which i would prefer to, but there seems to be no way
>> now
>>>>>>>> to do that).
>>>>>>>> 
>>>>>>>> So... after i run an instance of MR job, the hbase connection seems
>> to
>>>>>>>> be leaked. It also leaks zk connection , which is a problem since
>>>>>>>> zookeeper instances have limits on how many connections can be made
>>>>>>>> from the same IP and eventually the client is not able to create any
>>>>>>>> new HTables anymore since it can't establish any new zookeeper
>>>>>>>> connections.
>>>>>>>> 
>>>>>>>> I tried to do explicit cleanup by calling
>>>>>>>> HConnectionManager.deleteConnection (Configuration) passing in the
>>>>>>>> configuration that i used to create MR job. Doesn't seem to work.
>>>>>>>> 
>>>>>>>> So.. Is there a way to run MR job with TableInputFormat without
>>>>>>>> leaking a connection? I am pretty sure i am not creating any HTables
>>>>>>>> in the client side. Or is it a bug? I spent several days now
>>>>>>>> investigation an issue but i am still not able to come up with a
>>>>>>>> workaround against zookeeper connection leaks in HBase MR jobs.
>>>>>>>> 
>>>>>>>> thank you very much.
>>>>>>>> -Dmitriy
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>>> 
>>>>> --
>>>>> Todd Lipcon
>>>>> Software Engineer, Cloudera
>>>> 
>>>> 
>> 
>> 

Reply via email to