deleteAllConnections works well for my case..I can live with this but not with 
connection leaks
thanks for the idea

Venkatesh

 


 

 

-----Original Message-----
From: Ruben Quintero <rfq_...@yahoo.com>
To: user@hbase.apache.org
Sent: Wed, Apr 13, 2011 4:25 pm
Subject: Re: hbase -0.90.x upgrade - zookeeper exception in mapreduce job


Venkatesh, I guess the two quick and dirty solutions are:



- Call deleteAllConnections(bool) at the end of your MapReduce jobs, or 

periodically. If you have no other tables or pools, etc. open, then no problem. 

If you do, they'll start throwing IOExceptions, but you can re-instantiate them 

with a new config and then continue as usual. (You do have to change the config 

or it'll simply grab the closed, cached one from the HCM).



- As J-D said, subclasss TIF and basically copy the old setConf, except don't 

clone the conf that gets sent to the table.



Each one has a downside and are definitely not ideal, but if you either don't 

modify the config in your job or don't have any other important hbase 

connections, then you can use the appropriate one. 





Thanks for the assistance, J-D. It's great that these forums are active and 

helpful.



- Ruben









________________________________

From: Jean-Daniel Cryans <jdcry...@apache.org>

To: user@hbase.apache.org

Sent: Wed, April 13, 2011 3:50:42 PM

Subject: Re: hbase -0.90.x upgrade - zookeeper exception in mapreduce job



Yeah for a JVM running forever it won't work.



If you know for a fact that the configuration passed to TIF won't be

changed then you can subclass it and override setConf to not clone the

conf.



J-D



On Wed, Apr 13, 2011 at 12:45 PM, Ruben Quintero <rfq_...@yahoo.com> wrote:

> The problem is the connections are never closed... so they just keep piling up

> until it hits the max. My max is at 400 right now, so after 14-15 hours of

> running, it gets stuck in an endless connection retry.

>

> I saw that the HConnectionManager will kick older HConnections out, but the

> problem is that their ZooKeeper threads continue on. Those need to be 

>explicitly

> closed.

>

>

>

> Again, this is only an issue inside JVMs set to run forever, like Venkatesh

> said, because that's when the orphaned ZK connections will have a chance to

> build up to whatever your maximum is. Setting that higher and higher is just

> prolonging uptime before the eventual crash. It's essentially a memory

> (connection) leak within TableInputFormat, since there is no way that I can 

see

> to properly access and close those spawned connections.

>

> One question for you, JD: Inside of TableInputFormat.setConf, does the

> Configuration need to be cloned? (i.e. setHTable(new HTable(new

> Configuration(conf), tableName)); ). I'm guessing this is to prevent changes

> within the job from affecting the table and vice-versa...but if it weren't

> cloned, then you could use the job configuration (job.getConfiguration()) to

> close the connection....

>

> Other quick fixes that I can think of, none of which are very pretty:

> 1 - Just call deleteAllConnections(bool), and have any other processes using

> HConnections recover from that.

> 2 - Make the static HBASE_INSTANCES map accessible (public).... then you could

> iterate through open connections and try to match configs....

>

> Venkatesh - unless you have other processes in your JVM accessing HBase (I 

have

> one), #1 might be the best bet.

>

> - Ruben

>

>

>

> ________________________________

> From: Jean-Daniel Cryans <jdcry...@apache.org>

> To: user@hbase.apache.org

> Sent: Wed, April 13, 2011 3:22:48 PM

> Subject: Re: hbase -0.90.x upgrade - zookeeper exception in mapreduce job

>

> Like I said, it's a zookeeper configuration that you can change. If

> hbase is managing your zookeeper then set

> hbase.zookeeper.property.maxClientCnxns to something higher than 30

> and restart the zk server (can be done while hbase is running).

>

> J-D

>

> On Wed, Apr 13, 2011 at 12:04 PM, Venkatesh <vramanatha...@aol.com> wrote:

>> Reuben:

>> Yes..I've the exact same issue now..& I'm also kicking off from another jvm

>>that runs for ever..

>> I don't have an alternate solution..either modify hbase code (or) modify my

>>code to kick off

>> as a standalone jvm (or) hopefully 0.90.3 release soon :)

>> J-D/St.Ack may have some suggestions

>>

>>

>> V

>>

>>

>>

>>

>>

>>

>>

>>

>> -----Original Message-----

>> From: Ruben Quintero <rfq_...@yahoo.com>

>> To: user@hbase.apache.org

>> Sent: Wed, Apr 13, 2011 2:39 pm

>> Subject: Re: hbase -0.90.x upgrade - zookeeper exception in mapreduce job

>>

>>

>> 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 <vramanatha...@aol.com>

>>

>> To: user@hbase.apache.org

>>

>> 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 <jdcry...@apache.org>

>>

>> To: user@hbase.apache.org

>>

>> 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

>>

>>l

>>

>>

>>

>>

>>

>>

>>

>>

>>

>> J-D

>>

>>

>>

>>

>>

>>

>>

>> On Tue, Apr 12, 2011 at 3:09 PM, Ruben Quintero <rfq_...@yahoo.com> 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 <jdcry...@apache.org>

>>

>>

>>

>>> To: user@hbase.apache.org

>>

>>

>>

>>> 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 <vramanatha...@aol.com> 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