You could set the max connection for ZK much higher, we decided to do
that for 0.90.3:

https://issues.apache.org/jira/browse/HBASE-3773

Then in 0.92 we're going to change how we are managing the connections:

https://issues.apache.org/jira/browse/HBASE-3774

J-D

On Wed, Apr 13, 2011 at 11:39 AM, Ruben Quintero <[email protected]> wrote:
> The problem I'm having is in getting the conf that is used to init the table
> within TableInputFormat. That's the one that's leaving open ZK connections for
> me.
>
> Following the code through, TableInputFormat initializes its HTable with new
> Configuration(new JobConf(conf)), where conf is the config I pass in via job
> initiation. I don't see a way of getting the initalized TableInputFormat in
> order to then get its table and its config to be able to properly close that
> connection. Cloned configs don't appear to produce similar hashes, either. The
> only other option I'm left with is closing all connections, but that disrupts
> things across the board.
>
>
> For MapReduce jobs run in their own JVM, this wouldn't be much of an issue, as
> the connection would just be closed on completion, but in my case (our code
> triggers the jobs internally), they simply pile up until the ConnectionLoss 
> hits
> due to too many ZK connections.
>
> Am I missing a way to get that buried table's config, or another way to kill 
> the
> orphaned connections?
>
> - Ruben
>
>
>
> ________________________________
> From: Venkatesh <[email protected]>
> To: [email protected]
> Sent: Wed, April 13, 2011 10:20:50 AM
> Subject: Re: hbase -0.90.x upgrade - zookeeper exception in mapreduce job
>
> Thanks J-D
> I made sure to pass conf objects around in places where I was n't..
> will give it a try
>
>
>
>
>
>
>
>
>
>
> -----Original Message-----
> From: Jean-Daniel Cryans <[email protected]>
> To: [email protected]
> Sent: Tue, Apr 12, 2011 6:40 pm
> Subject: Re: hbase -0.90.x upgrade - zookeeper exception in mapreduce job
>
>
> Yes there are a few places like that. Also when you create new
>
> HTables, you should also close their connections (this is not done in
>
> htable.close).
>
>
>
> See HTable's javadoc which says:
>
>
>
> Instances of HTable passed the same Configuration instance will share
>
> connections to servers out on the cluster and to the zookeeper
>
> ensemble as well as caches of region locations. This is usually a
>
> *good* thing. This happens because they will all share the same
>
> underlying HConnection instance. See HConnectionManager for more on
>
> how this mechanism works.
>
>
>
> and it points to HCM which has more information:
>
> http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HConnectionManager.html
>
>
>
>
> J-D
>
>
>
> On Tue, Apr 12, 2011 at 3:09 PM, Ruben Quintero <[email protected]> wrote:
>
>> I'm running into the same issue, but did some poking around and it seems that
>
>> Zookeeper connections are being left open by an HBase internal.
>
>>
>
>> Basically, I'm running a mapreduce job within another program, and noticed in
>
>> the logs that every time the job is run, a connection is open, but I never 
>> see
>
>> it closed again. The connection is opened within the job.submit().
>
>>
>
>> I looked closer and checked the jstack after running it for just under an
>
> hour,
>
>> and sure enough there are a ton of Zookeeper threads just sitting there.
>
> Here's
>
>> a pastebin link: http://pastebin.com/MccEuvrc
>
>>
>
>> I'm running 0.90.0 right now.
>
>>
>
>> - Ruben
>
>>
>
>>
>
>>
>
>>
>
>>
>
>>
>
>> ________________________________
>
>> From: Jean-Daniel Cryans <[email protected]>
>
>> To: [email protected]
>
>> Sent: Tue, April 12, 2011 4:23:05 PM
>
>> Subject: Re: hbase -0.90.x upgrade - zookeeper exception in mapreduce job
>
>>
>
>> It's more in the vain of
>
>> https://issues.apache.org/jira/browse/HBASE-3755 and
>
>> https://issues.apache.org/jira/browse/HBASE-3771
>
>>
>
>> Basically 0.90 has a regression regarding the handling of zookeeper
>
>> connections that make it that you have to be super careful not to have
>
>> more than 30 per machine (each new Configuration is one new ZK
>
>> connection). Upping your zookeeper max connection config should get
>
>> rid of your issues since you only get it occasionally.
>
>>
>
>> J-D
>
>>
>
>> On Tue, Apr 12, 2011 at 7:59 AM, Venkatesh <[email protected]> wrote:
>
>>>
>
>>>  I get this occasionally..(not all the time)..Upgrading from 0.20.6 to 
>>> 0.90.2
>
>>> Is this issue same as this JIRA
>
>>> https://issues.apache.org/jira/browse/HBASE-3578
>
>>>
>
>>> I'm using HBaseConfiguration.create() & setting that in job
>
>>> thx
>
>>> v
>
>>>
>
>>>
>
>>>  2011-04-12 02:13:06,870 ERROR Timer-0
>
>>>org.apache.hadoop.hbase.mapreduce.TableInputFormat -
>
>>>org.apache.hadoop.hbase.ZooKeeperConnectionException:
>
>>>org.apache.hadoop.hbase.ZooKeeperConnectionException:
>
>>>org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode
>
> =
>
>>>ConnectionLoss for /hbase        at
>
>>>org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.getZooKeeperWatcher(HConnectionManager.java:1000)
>>>
>
>>>
>
>>>        at
>
>>>org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:303)
>>>
>
>>>
>
>>>        at
>
>>>org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.<init>(HConnectionManager.java:294)
>>>
>
>>>
>
>>>        at
>
>>>org.apache.hadoop.hbase.client.HConnectionManager.getConnection(HConnectionManager.java:156)
>>>
>
>>>        at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:167)
>
>>>        at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:145)
>
>>>        at
>
>>>org.apache.hadoop.hbase.mapreduce.TableInputFormat.setConf(TableInputFormat.java:91)
>>>
>
>>>
>
>>>        at
>
>>>org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:62)
>
>>>        at
>
>>>org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:117)
>
>>>        at
>
>> org.apache.hadoop.mapred.JobClient.writeNewSplits(JobClient.java:882)
>
>>>        at
>
>>>org.apache.hadoop.mapred.JobClient.submitJobInternal(JobClient.java:779)
>
>>>        at org.apache.hadoop.mapreduce.Job.submit(Job.java:432)
>
>>>        at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:448)
>
>>>
>
>>>
>
>>>
>
>>>
>
>>
>
>>
>
>>
>
>>
>
>
>

Reply via email to