You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by charan kumar <ch...@gmail.com> on 2011/02/03 20:47:29 UTC

Region Servers Crashing during Random Reads

Hello,

 I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2 CPU, 6 GB
RAM)

I had 9 Region Servers crash (out of 30) in a span of 30 minutes during a
heavy reads. It looks like a GC, ZooKeeper Connection Timeout thingy to me.
I did all recommended configuration from the Hbase wiki... Any other
suggestions?


2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew (promotion
failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
[CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]

2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew (promotion
failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
[CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]

2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew (promotion
failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
[CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]


The following is the log entry in region Server

2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn: Client session
timed out, have not heard from server in 47172ms for sessionid
0x12db9f722421ce3, closing socket connection and attempting reconnect
2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn: Client session
timed out, have not heard from server in 48159ms for sessionid
0x22db9f722501d93, closing socket connection and attempting reconnect
2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn: Opening socket
connection to server XXXXXXXXXXXXXXXX
2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn: Socket
connection established to XXXXXXXXX, initiating session
2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Opening socket
connection to server XXXXXXXXXXXXXXX
2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Socket
connection established to XXXXXXXXXXXXXXXXXXXXX, initiating session
2011-02-03 10:37:44,767 DEBUG
org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU eviction
started; Attempting to free 81.93 MB of total=696.25 MB
2011-02-03 10:37:44,784 DEBUG
org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU eviction
completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB, multi=309.77
MB, memory=0 KB
2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn: Unable to
reconnect to ZooKeeper service, session 0x22db9f722501d93 has expired,
closing socket connection
2011-02-03 10:37:45,206 INFO
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation:
This client just lost it's session with ZooKeeper, trying to reconnect.
2011-02-03 10:37:45,453 INFO
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation:
Trying to reconnect to zookeeper
2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn: Unable to
reconnect to ZooKeeper service, session 0x12db9f722421ce3 has expired,
closing socket connection
gionserver:60020-0x22db9f722501d93 regionserver:60020-0x22db9f722501d93
received expired from ZooKeeper, aborting
org.apache.zookeeper.KeeperException$SessionExpiredException:
KeeperErrorCode = Session expired
        at
org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(ZooKeeperWatcher.java:328)
        at
org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:246)
        at
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:530)
        at
org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
handled exception: org.apache.hadoop.hbase.YouAreDeadException: Server
REPORT rejected; currently processing XXXXXXXXXXXX,60020,1296684296172 as
dead server
org.apache.hadoop.hbase.YouAreDeadException:
org.apache.hadoop.hbase.YouAreDeadException: Server REPORT rejected;
currently processing XXXXXXXXXXXX,60020,1296684296172 as dead server
        at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
Method)
        at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
        at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
        at
org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:96)
        at
org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:80)
        at
org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerReport(HRegionServer.java:729)
        at
org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:586)
        at java.lang.Thread.run(Thread.java:619)


2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew (promotion
failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
[CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]



Thanks,
Charan

Re: Region Servers Crashing during Random Reads

Posted by Ryan Rawson <ry...@gmail.com>.
Under our load at su, the new gen would grow to max size and take 800+ ms. I
would consider setting the ms goal to 20-40ms (what we get in prod now). At
1gb par new i would expect large pauses. Plus in my previous tests the
promotion was like 75% even with a huge par new.

This is all based on my bulk loading and other prod workload from mid 2009.
The primary difference this year is just higher load :)
On Feb 4, 2011 8:21 AM, "Stack" <st...@duboce.net> wrote:
> On Fri, Feb 4, 2011 at 12:20 AM, Lars George <la...@gmail.com>
wrote:
>> I saw the -XX:MaxGCPauseMillis option too and assumed it is not that
>> effective as it was never suggested so far. So it was simply not tried
>> yet and someone has to be the guinea pig?
>>
>
> Yeah, haven't had good experience with these upper-bounds in pause
> time. We can add it though.
>
>
>> So that 6M NewSize in the Wiki was OK previously and now 256M is
>> actually better? The represent opposite ideas and I still try to
>> understand which is better now, small eden or reasonably large one?
>>
>
> This is best thing I've read on GC:
> http://www.scribd.com/doc/37127094/GCTuningPresentationFISL10
>
> As I read it, avoid promotion to old gen if at all possible.
>
>
>> Setting the -Xmn fixed seems to make sense given that it does do OK
>> with various heap size and a ration may be too liberal giving away
>> space.
>>
>
> 256M seems way small still given what I read above. Woud also like to
> avoid hardcoding so its 256M whether user runs with 1G -- the default
> or 8G. If we set NewRatio to 32, thats about 256M new gen in 8G heap
> (http://blogs.sun.com/jonthecollector/entry/the_second_most_important_gc).
> How about we set it to 16. Then young gen is about 5%.
>
> How about:
>
> -XX:+UseConcMarkSweepGC
> -XX:+UseParNewGC
> -XX:NewRatio=16
> -XX:CMSInitiatingOccupancyFraction=70
> -XX:+UseCMSInitiatingOccupancyOnly
> -XX:MaxGCPauseMillis=100
>
> We can add above to your auto-config patch Todd. Yes, Lars, would
> like to pull into the book but would also add note into hbase-env.sh
> to say this is just starting point .... and we'd let go of the wiki
> (it needs updating to reference Todd's new LZO fixings too). If above
> looks reasonable, I'll test it.
>
> St.Ack
>
> St.Ack

Re: Region Servers Crashing during Random Reads

Posted by Stack <st...@duboce.net>.
On Fri, Feb 4, 2011 at 12:20 AM, Lars George <la...@gmail.com> wrote:
> I saw the -XX:MaxGCPauseMillis option too and assumed it is not that
> effective as it was never suggested so far. So it was simply not tried
> yet and someone has to be the guinea pig?
>

Yeah, haven't had good experience with these upper-bounds in pause
time.  We can add it though.


> So that 6M NewSize in the Wiki was OK previously and now 256M is
> actually better? The represent opposite ideas and I still try to
> understand which is better now, small eden or reasonably large one?
>

This is best thing I've read on GC:
http://www.scribd.com/doc/37127094/GCTuningPresentationFISL10

As I read it, avoid promotion to old gen if at all possible.


> Setting the -Xmn fixed seems to make sense given that it does do OK
> with various heap size and a ration may be too liberal giving away
> space.
>

256M seems way small still given what I read above.  Woud also like to
avoid hardcoding so its 256M whether user runs with 1G -- the default
or 8G.  If we set NewRatio to 32, thats about 256M new gen in 8G heap
(http://blogs.sun.com/jonthecollector/entry/the_second_most_important_gc).
 How about we set it to 16.  Then young gen is about 5%.

How about:

-XX:+UseConcMarkSweepGC
-XX:+UseParNewGC
-XX:NewRatio=16
-XX:CMSInitiatingOccupancyFraction=70
-XX:+UseCMSInitiatingOccupancyOnly
-XX:MaxGCPauseMillis=100

We can add above to your auto-config patch Todd.  Yes, Lars, would
like to pull into the book but would also add note into hbase-env.sh
to say this is just starting point .... and we'd let go of the wiki
(it needs updating to reference Todd's new LZO fixings too).  If above
looks reasonable, I'll test it.

St.Ack

St.Ack

Re: Region Servers Crashing during Random Reads

Posted by Lars George <la...@gmail.com>.
I saw the -XX:MaxGCPauseMillis option too and assumed it is not that
effective as it was never suggested so far. So it was simply not tried
yet and someone has to be the guinea pig?

So that 6M NewSize in the Wiki was OK previously and now 256M is
actually better? The represent opposite ideas and I still try to
understand which is better now, small eden or reasonably large one?

Setting the -Xmn fixed seems to make sense given that it does do OK
with various heap size and a ration may be too liberal giving away
space.

PS: Just interested and as I said earlier relying on your expertise

Lars

On Fri, Feb 4, 2011 at 9:07 AM, Todd Lipcon <to...@cloudera.com> wrote:
> Did some reading through the hotspot source to look into this. In the past
> I've advised small newsize to keep the pause times down on the minor
> collections, but after looking at the source I think there's a better way:
> -XX:MaxGCPauseMillis=100  (or whatever pause seems reasonable to you). If
> you don't specify this, the adaptive size policy will try to go for
> throughput with no regard for pause times. This may be fine if you're
> entirely in a batch scenario. If you have serving SLAs it's probably bad
> (Ryan said he's seen 400ms+ young gen collections).
>
> I wouldn't ever want the young gen to be more than 20% of the heap. We
> allocate 60% of the heap by default to cache and memstore, and CMS needs its
> wiggle room in the old gen.
>
> So, if you're in a backend/batch scenario where half second pauses a couple
> times a minute are no problem, I'd set MaxNewSize to maybe 15-20% of the
> heap, and then let the adaptive size policy tune for throughput.
> If you really want low latency, I'd do the above to set MaxNewSize but also
> set MaxGCPauseMillis to whatever your goal is.
>
> Note this hasn't been tested in production - the scary thing about adaptive
> policies is they can get used to one load profile and then not react quickly
> enough when the load profile changes (eg you start running an MR job). Might
> be best to use the adaptive policies in some load tests, see what kind of
> new size it picks, and then hard code that with -Xmn from then on.
>
> Or if you just want a short answer with little tuning, 256m in a 6-8G total
> heap seems to work well for me with 20-40ms young gen pauses when under
> load.
>
> -Todd
>
> On Thu, Feb 3, 2011 at 11:36 PM, Lars George <la...@gmail.com> wrote:
>
>> Hi Stack,
>>
>> I was just asking Todd the same thing, ie. fixed new size vs NewRatio.
>> He and you have done way more on GC debugging than me so I trust
>> whatever Todd or you say. I would leave the UseParNewGC for good
>> measure (not relying on implicit defaults). I also re-read just before
>> I saw your reply the HotSpot docs on GC performance optimizations and
>> also stopped at the UseCMSInitiatingOccupancyOnly option, wondering if
>> that is a good one to add. Again your call but sounds reasonable.
>>
>> This is much more current overall compared to what the wiki says so I
>> would change it there. I would also ask to maybe move this into the
>> book, can then evolve with the releases as opposed to the ugly Wiki
>> which is like an ulcer most of the time. If you are going to I also
>> recommend removing the old content and redirect to book. I know the
>> issue is Wiki can be changed whenever and the book can not, but I
>> would look at each page carefully and apply a GC promoting strategy,
>> ie. promote some of them in the tenured (i.e. the book) space.
>>
>> Lars
>>
>> On Fri, Feb 4, 2011 at 8:11 AM, Stack <st...@duboce.net> wrote:
>> > Yeah, our wiki page seems way off to me.  I can update it.  Rather
>> > than hardcoding absolute new gen size Todd, how about using
>> > -XX:NewRatio=3 say; i.e. 1/4 of heap is new gen (maybe it should be
>> > 1/3rd!).  Does UseParNewGC do anything?  I seem to see the 'parallel'
>> > rescans whether its on or off (This says its off by default,
>> > http://www.md.pp.ru/~eu/jdk6options.html#UseParNewGC, but I trust my
>> > eyes and this more
>> > http://blogs.sun.com/jonthecollector/category/Java).  70% for
>> > initiating fraction seems conservative (but I know what you are going
>> > to say and yes, you are right we should be conservative....).  We
>> > should tag on '-XX:+UseCMSInitiatingOccupancyOnly' too?
>> >
>> > If you are good w/ above changes (I can leave UseParNewGC in the mix),
>> > I'll make the changes to the wiki.
>> >
>> > Good stuff,
>> > St.Ack
>> >
>> >
>> > On Thu, Feb 3, 2011 at 10:26 PM, charan kumar <ch...@gmail.com>
>> wrote:
>> >> Here you go..
>> >>
>> >> HBase Performance tuning page
>> >> http://wiki.apache.org/hadoop/Hbase/FAQ#A7refers to the following
>> >> hadoop URL.
>> >>
>> >> http://wiki.apache.org/hadoop/PerformanceTuning
>> >>
>> >> Thanks,
>> >> Charan
>> >>
>> >>
>> >> On Thu, Feb 3, 2011 at 10:22 PM, Todd Lipcon <to...@cloudera.com> wrote:
>> >>
>> >>> Does the wiki really recommend that? Got a link handy?
>> >>>
>> >>> On Thu, Feb 3, 2011 at 10:20 PM, charan kumar <charan.kumar@gmail.com
>> >>> >wrote:
>> >>>
>> >>> > Todd,
>> >>> >
>> >>> >  That did the trick.  I think the wiki should be updated as well, no
>> >>> point
>> >>> > in recommending ParNew 6M or is it?
>> >>> >
>> >>> > Thanks,
>> >>> > Charan.
>> >>> >
>> >>> > On Thu, Feb 3, 2011 at 2:06 PM, Charan K <ch...@gmail.com>
>> wrote:
>> >>> >
>> >>> > > Thanks Todd.. I will try it out ..
>> >>> > >
>> >>> > >
>> >>> > > On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:
>> >>> > >
>> >>> > > > Hi Charan,
>> >>> > > >
>> >>> > > > Your GC settings are way off - 6m newsize will promote way too
>> much
>> >>> to
>> >>> > > the
>> >>> > > > oldgen.
>> >>> > > >
>> >>> > > > Try this:
>> >>> > > >
>> >>> > > > -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
>> >>> > > > -XX:CMSInitiatingOccupancyFraction=70
>> >>> > > >
>> >>> > > > -Todd
>> >>> > > >
>> >>> > > > On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <
>> >>> charan.kumar@gmail.com
>> >>> > > >wrote:
>> >>> > > >
>> >>> > > >> HI Jonathan,
>> >>> > > >>
>> >>> > > >> Thanks for you quick reply..
>> >>> > > >>
>> >>> > > >> Heap is set to 4G.
>> >>> > > >>
>> >>> > > >> Following are the JVM opts.
>> >>> > > >> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
>> >>> > > >> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
>> >>> > > >> -XX:MaxNewSize=6m"
>> >>> > > >>
>> >>> > > >> Are there any other options apart from increasing the RAM?
>> >>> > > >>
>> >>> > > >> I am adding some more info about the app.
>> >>> > > >>
>> >>> > > >>> We are storing web page data in HBase.
>> >>> > > >>> Row key is Hashed URL, for random distribution, since we dont
>> plan
>> >>> to
>> >>> > > do
>> >>> > > >> scan's..
>> >>> > > >>> We have LZOCompression Set on this column family.
>> >>> > > >>> We were noticing 1500 Reads, when reading the page content.
>> >>> > > >>> We have a column family, which stores just metadata of the page
>> >>> > "title"
>> >>> > > >> etc... When reading this the performance is whopping 12000 TPS.
>> >>> > > >>
>> >>> > > >> We though the issue could be because of N/w bandwidth used
>> between
>> >>> > HBase
>> >>> > > >> and Clients. So we disable LZO Compression on Column Family and
>> >>> > started
>> >>> > > >> doing the compression of the raw page on the client and
>> decompress
>> >>> it
>> >>> > > when
>> >>> > > >> readind (LZO).
>> >>> > > >>
>> >>> > > >>> With this my write performance jumped up from 2000 to 5000 at
>> peak.
>> >>> > > >>> With this approach, the servers are crashing... Not sure , why
>> only
>> >>> > > >> after
>> >>> > > >> turning of LZO... and doing the same from client.
>> >>> > > >>
>> >>> > > >>
>> >>> > > >>
>> >>> > > >> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com>
>> >>> wrote:
>> >>> > > >>
>> >>> > > >>> How much heap are you running on your RegionServers?
>> >>> > > >>>
>> >>> > > >>> 6GB of total RAM is on the low end.  For high throughput
>> >>> > applications,
>> >>> > > I
>> >>> > > >>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
>> >>> > > >>>
>> >>> > > >>>> -----Original Message-----
>> >>> > > >>>> From: charan kumar [mailto:charan.kumar@gmail.com]
>> >>> > > >>>> Sent: Thursday, February 03, 2011 11:47 AM
>> >>> > > >>>> To: user@hbase.apache.org
>> >>> > > >>>> Subject: Region Servers Crashing during Random Reads
>> >>> > > >>>>
>> >>> > > >>>> Hello,
>> >>> > > >>>>
>> >>> > > >>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2
>> >>> CPU,
>> >>> > 6
>> >>> > > >> GB
>> >>> > > >>>> RAM)
>> >>> > > >>>>
>> >>> > > >>>> I had 9 Region Servers crash (out of 30) in a span of 30
>> minutes
>> >>> > > during
>> >>> > > >> a
>> >>> > > >>> heavy
>> >>> > > >>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy
>> to
>> >>> > me.
>> >>> > > >>>> I did all recommended configuration from the Hbase wiki... Any
>> >>> other
>> >>> > > >>>> suggestions?
>> >>> > > >>>>
>> >>> > > >>>>
>> >>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632:
>> [ParNew
>> >>> > > >>>> (promotion
>> >>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>> >>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606:
>> [CMS-concurrent-mark:
>> >>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31
>> secs]
>> >>> > > >>>>
>> >>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785:
>> [ParNew
>> >>> > > >>>> (promotion
>> >>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>> >>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731:
>> [CMS-concurrent-mark:
>> >>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60
>> secs]
>> >>> > > >>>>
>> >>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632:
>> [ParNew
>> >>> > > >>>> (promotion
>> >>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>> >>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606:
>> [CMS-concurrent-mark:
>> >>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31
>> secs]
>> >>> > > >>>>
>> >>> > > >>>>
>> >>> > > >>>> The following is the log entry in region Server
>> >>> > > >>>>
>> >>> > > >>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn:
>> >>> Client
>> >>> > > >>>> session timed out, have not heard from server in 47172ms for
>> >>> > sessionid
>> >>> > > >>>> 0x12db9f722421ce3, closing socket connection and attempting
>> >>> > reconnect
>> >>> > > >>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn:
>> >>> Client
>> >>> > > >>>> session timed out, have not heard from server in 48159ms for
>> >>> > sessionid
>> >>> > > >>>> 0x22db9f722501d93, closing socket connection and attempting
>> >>> > reconnect
>> >>> > > >>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn:
>> >>> > Opening
>> >>> > > >>>> socket connection to server XXXXXXXXXXXXXXXX
>> >>> > > >>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn:
>> >>> Socket
>> >>> > > >>>> connection established to XXXXXXXXX, initiating session
>> >>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
>> >>> > Opening
>> >>> > > >>>> socket connection to server XXXXXXXXXXXXXXX
>> >>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
>> >>> Socket
>> >>> > > >>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating
>> >>> session
>> >>> > > >>>> 2011-02-03 10:37:44,767 DEBUG
>> >>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache
>> LRU
>> >>> > > >> eviction
>> >>> > > >>>> started; Attempting to free 81.93 MB of total=696.25 MB
>> >>> > > >>>> 2011-02-03 10:37:44,784 DEBUG
>> >>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache
>> LRU
>> >>> > > >> eviction
>> >>> > > >>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
>> >>> > > >>>> multi=309.77 MB, memory=0 KB
>> >>> > > >>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn:
>> >>> Unable
>> >>> > > to
>> >>> > > >>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has
>> >>> > expired,
>> >>> > > >>>> closing socket connection
>> >>> > > >>>> 2011-02-03 10:37:45,206 INFO
>> >>> > > >>>>
>> >>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>> >>> > > >>>> entation:
>> >>> > > >>>> This client just lost it's session with ZooKeeper, trying to
>> >>> > > reconnect.
>> >>> > > >>>> 2011-02-03 10:37:45,453 INFO
>> >>> > > >>>>
>> >>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>> >>> > > >>>> entation:
>> >>> > > >>>> Trying to reconnect to zookeeper
>> >>> > > >>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn:
>> >>> Unable
>> >>> > > to
>> >>> > > >>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has
>> >>> > expired,
>> >>> > > >>>> closing socket connection
>> >>> > > >>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
>> >>> > > >>>> 0x22db9f722501d93
>> >>> > > >>>> received expired from ZooKeeper, aborting
>> >>> > > >>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
>> >>> > > >>>> KeeperErrorCode = Session expired
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
>> >>> > > >>>> ZooKeeperWatcher.java:328)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
>> >>> > > >>>> erWatcher.java:246)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> >
>> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
>> >>> > > >>>> va:530)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
>> >>> > > >>>> handled exception:
>> org.apache.hadoop.hbase.YouAreDeadException:
>> >>> > Server
>> >>> > > >>>> REPORT rejected; currently processing
>> >>> > XXXXXXXXXXXX,60020,1296684296172
>> >>> > > >>>> as dead server
>> >>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException:
>> >>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT
>> >>> rejected;
>> >>> > > >>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead
>> >>> server
>> >>> > > >>>>        at
>> >>> > > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
>> >>> > > >>>> Method)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> > >
>> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
>> >>> > > >>>> AccessorImpl.java:39)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> > >
>> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
>> >>> > > >>>> structorAccessorImpl.java:27)
>> >>> > > >>>>        at
>> >>> > > >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> > org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
>> >>> > > >>>> ption.java:96)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
>> >>> > > >>>> Exception.java:80)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> > org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
>> >>> > > >>>> ort(HRegionServer.java:729)
>> >>> > > >>>>        at
>> >>> > > >>>>
>> >>> >
>> org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
>> >>> > > >>>> ava:586)
>> >>> > > >>>>        at java.lang.Thread.run(Thread.java:619)
>> >>> > > >>>>
>> >>> > > >>>>
>> >>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785:
>> [ParNew
>> >>> > > >>>> (promotion
>> >>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>> >>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731:
>> [CMS-concurrent-mark:
>> >>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60
>> secs]
>> >>> > > >>>>
>> >>> > > >>>>
>> >>> > > >>>>
>> >>> > > >>>> Thanks,
>> >>> > > >>>> Charan
>> >>> > > >>>
>> >>> > > >>
>> >>> > > >
>> >>> > > >
>> >>> > > >
>> >>> > > > --
>> >>> > > > Todd Lipcon
>> >>> > > > Software Engineer, Cloudera
>> >>> > >
>> >>> >
>> >>>
>> >>>
>> >>>
>> >>> --
>> >>> Todd Lipcon
>> >>> Software Engineer, Cloudera
>> >>>
>> >>
>> >
>>
>
>
>
> --
> Todd Lipcon
> Software Engineer, Cloudera
>

Re: Region Servers Crashing during Random Reads

Posted by Todd Lipcon <to...@cloudera.com>.
Did some reading through the hotspot source to look into this. In the past
I've advised small newsize to keep the pause times down on the minor
collections, but after looking at the source I think there's a better way:
-XX:MaxGCPauseMillis=100  (or whatever pause seems reasonable to you). If
you don't specify this, the adaptive size policy will try to go for
throughput with no regard for pause times. This may be fine if you're
entirely in a batch scenario. If you have serving SLAs it's probably bad
(Ryan said he's seen 400ms+ young gen collections).

I wouldn't ever want the young gen to be more than 20% of the heap. We
allocate 60% of the heap by default to cache and memstore, and CMS needs its
wiggle room in the old gen.

So, if you're in a backend/batch scenario where half second pauses a couple
times a minute are no problem, I'd set MaxNewSize to maybe 15-20% of the
heap, and then let the adaptive size policy tune for throughput.
If you really want low latency, I'd do the above to set MaxNewSize but also
set MaxGCPauseMillis to whatever your goal is.

Note this hasn't been tested in production - the scary thing about adaptive
policies is they can get used to one load profile and then not react quickly
enough when the load profile changes (eg you start running an MR job). Might
be best to use the adaptive policies in some load tests, see what kind of
new size it picks, and then hard code that with -Xmn from then on.

Or if you just want a short answer with little tuning, 256m in a 6-8G total
heap seems to work well for me with 20-40ms young gen pauses when under
load.

-Todd

On Thu, Feb 3, 2011 at 11:36 PM, Lars George <la...@gmail.com> wrote:

> Hi Stack,
>
> I was just asking Todd the same thing, ie. fixed new size vs NewRatio.
> He and you have done way more on GC debugging than me so I trust
> whatever Todd or you say. I would leave the UseParNewGC for good
> measure (not relying on implicit defaults). I also re-read just before
> I saw your reply the HotSpot docs on GC performance optimizations and
> also stopped at the UseCMSInitiatingOccupancyOnly option, wondering if
> that is a good one to add. Again your call but sounds reasonable.
>
> This is much more current overall compared to what the wiki says so I
> would change it there. I would also ask to maybe move this into the
> book, can then evolve with the releases as opposed to the ugly Wiki
> which is like an ulcer most of the time. If you are going to I also
> recommend removing the old content and redirect to book. I know the
> issue is Wiki can be changed whenever and the book can not, but I
> would look at each page carefully and apply a GC promoting strategy,
> ie. promote some of them in the tenured (i.e. the book) space.
>
> Lars
>
> On Fri, Feb 4, 2011 at 8:11 AM, Stack <st...@duboce.net> wrote:
> > Yeah, our wiki page seems way off to me.  I can update it.  Rather
> > than hardcoding absolute new gen size Todd, how about using
> > -XX:NewRatio=3 say; i.e. 1/4 of heap is new gen (maybe it should be
> > 1/3rd!).  Does UseParNewGC do anything?  I seem to see the 'parallel'
> > rescans whether its on or off (This says its off by default,
> > http://www.md.pp.ru/~eu/jdk6options.html#UseParNewGC, but I trust my
> > eyes and this more
> > http://blogs.sun.com/jonthecollector/category/Java).  70% for
> > initiating fraction seems conservative (but I know what you are going
> > to say and yes, you are right we should be conservative....).  We
> > should tag on '-XX:+UseCMSInitiatingOccupancyOnly' too?
> >
> > If you are good w/ above changes (I can leave UseParNewGC in the mix),
> > I'll make the changes to the wiki.
> >
> > Good stuff,
> > St.Ack
> >
> >
> > On Thu, Feb 3, 2011 at 10:26 PM, charan kumar <ch...@gmail.com>
> wrote:
> >> Here you go..
> >>
> >> HBase Performance tuning page
> >> http://wiki.apache.org/hadoop/Hbase/FAQ#A7refers to the following
> >> hadoop URL.
> >>
> >> http://wiki.apache.org/hadoop/PerformanceTuning
> >>
> >> Thanks,
> >> Charan
> >>
> >>
> >> On Thu, Feb 3, 2011 at 10:22 PM, Todd Lipcon <to...@cloudera.com> wrote:
> >>
> >>> Does the wiki really recommend that? Got a link handy?
> >>>
> >>> On Thu, Feb 3, 2011 at 10:20 PM, charan kumar <charan.kumar@gmail.com
> >>> >wrote:
> >>>
> >>> > Todd,
> >>> >
> >>> >  That did the trick.  I think the wiki should be updated as well, no
> >>> point
> >>> > in recommending ParNew 6M or is it?
> >>> >
> >>> > Thanks,
> >>> > Charan.
> >>> >
> >>> > On Thu, Feb 3, 2011 at 2:06 PM, Charan K <ch...@gmail.com>
> wrote:
> >>> >
> >>> > > Thanks Todd.. I will try it out ..
> >>> > >
> >>> > >
> >>> > > On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:
> >>> > >
> >>> > > > Hi Charan,
> >>> > > >
> >>> > > > Your GC settings are way off - 6m newsize will promote way too
> much
> >>> to
> >>> > > the
> >>> > > > oldgen.
> >>> > > >
> >>> > > > Try this:
> >>> > > >
> >>> > > > -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
> >>> > > > -XX:CMSInitiatingOccupancyFraction=70
> >>> > > >
> >>> > > > -Todd
> >>> > > >
> >>> > > > On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <
> >>> charan.kumar@gmail.com
> >>> > > >wrote:
> >>> > > >
> >>> > > >> HI Jonathan,
> >>> > > >>
> >>> > > >> Thanks for you quick reply..
> >>> > > >>
> >>> > > >> Heap is set to 4G.
> >>> > > >>
> >>> > > >> Following are the JVM opts.
> >>> > > >> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
> >>> > > >> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
> >>> > > >> -XX:MaxNewSize=6m"
> >>> > > >>
> >>> > > >> Are there any other options apart from increasing the RAM?
> >>> > > >>
> >>> > > >> I am adding some more info about the app.
> >>> > > >>
> >>> > > >>> We are storing web page data in HBase.
> >>> > > >>> Row key is Hashed URL, for random distribution, since we dont
> plan
> >>> to
> >>> > > do
> >>> > > >> scan's..
> >>> > > >>> We have LZOCompression Set on this column family.
> >>> > > >>> We were noticing 1500 Reads, when reading the page content.
> >>> > > >>> We have a column family, which stores just metadata of the page
> >>> > "title"
> >>> > > >> etc... When reading this the performance is whopping 12000 TPS.
> >>> > > >>
> >>> > > >> We though the issue could be because of N/w bandwidth used
> between
> >>> > HBase
> >>> > > >> and Clients. So we disable LZO Compression on Column Family and
> >>> > started
> >>> > > >> doing the compression of the raw page on the client and
> decompress
> >>> it
> >>> > > when
> >>> > > >> readind (LZO).
> >>> > > >>
> >>> > > >>> With this my write performance jumped up from 2000 to 5000 at
> peak.
> >>> > > >>> With this approach, the servers are crashing... Not sure , why
> only
> >>> > > >> after
> >>> > > >> turning of LZO... and doing the same from client.
> >>> > > >>
> >>> > > >>
> >>> > > >>
> >>> > > >> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com>
> >>> wrote:
> >>> > > >>
> >>> > > >>> How much heap are you running on your RegionServers?
> >>> > > >>>
> >>> > > >>> 6GB of total RAM is on the low end.  For high throughput
> >>> > applications,
> >>> > > I
> >>> > > >>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
> >>> > > >>>
> >>> > > >>>> -----Original Message-----
> >>> > > >>>> From: charan kumar [mailto:charan.kumar@gmail.com]
> >>> > > >>>> Sent: Thursday, February 03, 2011 11:47 AM
> >>> > > >>>> To: user@hbase.apache.org
> >>> > > >>>> Subject: Region Servers Crashing during Random Reads
> >>> > > >>>>
> >>> > > >>>> Hello,
> >>> > > >>>>
> >>> > > >>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2
> >>> CPU,
> >>> > 6
> >>> > > >> GB
> >>> > > >>>> RAM)
> >>> > > >>>>
> >>> > > >>>> I had 9 Region Servers crash (out of 30) in a span of 30
> minutes
> >>> > > during
> >>> > > >> a
> >>> > > >>> heavy
> >>> > > >>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy
> to
> >>> > me.
> >>> > > >>>> I did all recommended configuration from the Hbase wiki... Any
> >>> other
> >>> > > >>>> suggestions?
> >>> > > >>>>
> >>> > > >>>>
> >>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632:
> [ParNew
> >>> > > >>>> (promotion
> >>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> >>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606:
> [CMS-concurrent-mark:
> >>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31
> secs]
> >>> > > >>>>
> >>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785:
> [ParNew
> >>> > > >>>> (promotion
> >>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> >>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731:
> [CMS-concurrent-mark:
> >>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60
> secs]
> >>> > > >>>>
> >>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632:
> [ParNew
> >>> > > >>>> (promotion
> >>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> >>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606:
> [CMS-concurrent-mark:
> >>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31
> secs]
> >>> > > >>>>
> >>> > > >>>>
> >>> > > >>>> The following is the log entry in region Server
> >>> > > >>>>
> >>> > > >>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn:
> >>> Client
> >>> > > >>>> session timed out, have not heard from server in 47172ms for
> >>> > sessionid
> >>> > > >>>> 0x12db9f722421ce3, closing socket connection and attempting
> >>> > reconnect
> >>> > > >>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn:
> >>> Client
> >>> > > >>>> session timed out, have not heard from server in 48159ms for
> >>> > sessionid
> >>> > > >>>> 0x22db9f722501d93, closing socket connection and attempting
> >>> > reconnect
> >>> > > >>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn:
> >>> > Opening
> >>> > > >>>> socket connection to server XXXXXXXXXXXXXXXX
> >>> > > >>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn:
> >>> Socket
> >>> > > >>>> connection established to XXXXXXXXX, initiating session
> >>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
> >>> > Opening
> >>> > > >>>> socket connection to server XXXXXXXXXXXXXXX
> >>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
> >>> Socket
> >>> > > >>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating
> >>> session
> >>> > > >>>> 2011-02-03 10:37:44,767 DEBUG
> >>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache
> LRU
> >>> > > >> eviction
> >>> > > >>>> started; Attempting to free 81.93 MB of total=696.25 MB
> >>> > > >>>> 2011-02-03 10:37:44,784 DEBUG
> >>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache
> LRU
> >>> > > >> eviction
> >>> > > >>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
> >>> > > >>>> multi=309.77 MB, memory=0 KB
> >>> > > >>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn:
> >>> Unable
> >>> > > to
> >>> > > >>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has
> >>> > expired,
> >>> > > >>>> closing socket connection
> >>> > > >>>> 2011-02-03 10:37:45,206 INFO
> >>> > > >>>>
> >>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> >>> > > >>>> entation:
> >>> > > >>>> This client just lost it's session with ZooKeeper, trying to
> >>> > > reconnect.
> >>> > > >>>> 2011-02-03 10:37:45,453 INFO
> >>> > > >>>>
> >>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> >>> > > >>>> entation:
> >>> > > >>>> Trying to reconnect to zookeeper
> >>> > > >>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn:
> >>> Unable
> >>> > > to
> >>> > > >>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has
> >>> > expired,
> >>> > > >>>> closing socket connection
> >>> > > >>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
> >>> > > >>>> 0x22db9f722501d93
> >>> > > >>>> received expired from ZooKeeper, aborting
> >>> > > >>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
> >>> > > >>>> KeeperErrorCode = Session expired
> >>> > > >>>>        at
> >>> > > >>>>
> >>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
> >>> > > >>>> ZooKeeperWatcher.java:328)
> >>> > > >>>>        at
> >>> > > >>>>
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
> >>> > > >>>> erWatcher.java:246)
> >>> > > >>>>        at
> >>> > > >>>>
> >>> >
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
> >>> > > >>>> va:530)
> >>> > > >>>>        at
> >>> > > >>>>
> >>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
> >>> > > >>>> handled exception:
> org.apache.hadoop.hbase.YouAreDeadException:
> >>> > Server
> >>> > > >>>> REPORT rejected; currently processing
> >>> > XXXXXXXXXXXX,60020,1296684296172
> >>> > > >>>> as dead server
> >>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException:
> >>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT
> >>> rejected;
> >>> > > >>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead
> >>> server
> >>> > > >>>>        at
> >>> > > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> >>> > > >>>> Method)
> >>> > > >>>>        at
> >>> > > >>>>
> >>> > >
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
> >>> > > >>>> AccessorImpl.java:39)
> >>> > > >>>>        at
> >>> > > >>>>
> >>> > >
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
> >>> > > >>>> structorAccessorImpl.java:27)
> >>> > > >>>>        at
> >>> > > >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
> >>> > > >>>>        at
> >>> > > >>>>
> >>> > org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
> >>> > > >>>> ption.java:96)
> >>> > > >>>>        at
> >>> > > >>>>
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
> >>> > > >>>> Exception.java:80)
> >>> > > >>>>        at
> >>> > > >>>>
> >>> > org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
> >>> > > >>>> ort(HRegionServer.java:729)
> >>> > > >>>>        at
> >>> > > >>>>
> >>> >
> org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
> >>> > > >>>> ava:586)
> >>> > > >>>>        at java.lang.Thread.run(Thread.java:619)
> >>> > > >>>>
> >>> > > >>>>
> >>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785:
> [ParNew
> >>> > > >>>> (promotion
> >>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> >>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731:
> [CMS-concurrent-mark:
> >>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60
> secs]
> >>> > > >>>>
> >>> > > >>>>
> >>> > > >>>>
> >>> > > >>>> Thanks,
> >>> > > >>>> Charan
> >>> > > >>>
> >>> > > >>
> >>> > > >
> >>> > > >
> >>> > > >
> >>> > > > --
> >>> > > > Todd Lipcon
> >>> > > > Software Engineer, Cloudera
> >>> > >
> >>> >
> >>>
> >>>
> >>>
> >>> --
> >>> Todd Lipcon
> >>> Software Engineer, Cloudera
> >>>
> >>
> >
>



-- 
Todd Lipcon
Software Engineer, Cloudera

Re: Region Servers Crashing during Random Reads

Posted by Lars George <la...@gmail.com>.
Hi Stack,

I was just asking Todd the same thing, ie. fixed new size vs NewRatio.
He and you have done way more on GC debugging than me so I trust
whatever Todd or you say. I would leave the UseParNewGC for good
measure (not relying on implicit defaults). I also re-read just before
I saw your reply the HotSpot docs on GC performance optimizations and
also stopped at the UseCMSInitiatingOccupancyOnly option, wondering if
that is a good one to add. Again your call but sounds reasonable.

This is much more current overall compared to what the wiki says so I
would change it there. I would also ask to maybe move this into the
book, can then evolve with the releases as opposed to the ugly Wiki
which is like an ulcer most of the time. If you are going to I also
recommend removing the old content and redirect to book. I know the
issue is Wiki can be changed whenever and the book can not, but I
would look at each page carefully and apply a GC promoting strategy,
ie. promote some of them in the tenured (i.e. the book) space.

Lars

On Fri, Feb 4, 2011 at 8:11 AM, Stack <st...@duboce.net> wrote:
> Yeah, our wiki page seems way off to me.  I can update it.  Rather
> than hardcoding absolute new gen size Todd, how about using
> -XX:NewRatio=3 say; i.e. 1/4 of heap is new gen (maybe it should be
> 1/3rd!).  Does UseParNewGC do anything?  I seem to see the 'parallel'
> rescans whether its on or off (This says its off by default,
> http://www.md.pp.ru/~eu/jdk6options.html#UseParNewGC, but I trust my
> eyes and this more
> http://blogs.sun.com/jonthecollector/category/Java).  70% for
> initiating fraction seems conservative (but I know what you are going
> to say and yes, you are right we should be conservative....).  We
> should tag on '-XX:+UseCMSInitiatingOccupancyOnly' too?
>
> If you are good w/ above changes (I can leave UseParNewGC in the mix),
> I'll make the changes to the wiki.
>
> Good stuff,
> St.Ack
>
>
> On Thu, Feb 3, 2011 at 10:26 PM, charan kumar <ch...@gmail.com> wrote:
>> Here you go..
>>
>> HBase Performance tuning page
>> http://wiki.apache.org/hadoop/Hbase/FAQ#A7refers to the following
>> hadoop URL.
>>
>> http://wiki.apache.org/hadoop/PerformanceTuning
>>
>> Thanks,
>> Charan
>>
>>
>> On Thu, Feb 3, 2011 at 10:22 PM, Todd Lipcon <to...@cloudera.com> wrote:
>>
>>> Does the wiki really recommend that? Got a link handy?
>>>
>>> On Thu, Feb 3, 2011 at 10:20 PM, charan kumar <charan.kumar@gmail.com
>>> >wrote:
>>>
>>> > Todd,
>>> >
>>> >  That did the trick.  I think the wiki should be updated as well, no
>>> point
>>> > in recommending ParNew 6M or is it?
>>> >
>>> > Thanks,
>>> > Charan.
>>> >
>>> > On Thu, Feb 3, 2011 at 2:06 PM, Charan K <ch...@gmail.com> wrote:
>>> >
>>> > > Thanks Todd.. I will try it out ..
>>> > >
>>> > >
>>> > > On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:
>>> > >
>>> > > > Hi Charan,
>>> > > >
>>> > > > Your GC settings are way off - 6m newsize will promote way too much
>>> to
>>> > > the
>>> > > > oldgen.
>>> > > >
>>> > > > Try this:
>>> > > >
>>> > > > -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
>>> > > > -XX:CMSInitiatingOccupancyFraction=70
>>> > > >
>>> > > > -Todd
>>> > > >
>>> > > > On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <
>>> charan.kumar@gmail.com
>>> > > >wrote:
>>> > > >
>>> > > >> HI Jonathan,
>>> > > >>
>>> > > >> Thanks for you quick reply..
>>> > > >>
>>> > > >> Heap is set to 4G.
>>> > > >>
>>> > > >> Following are the JVM opts.
>>> > > >> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
>>> > > >> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
>>> > > >> -XX:MaxNewSize=6m"
>>> > > >>
>>> > > >> Are there any other options apart from increasing the RAM?
>>> > > >>
>>> > > >> I am adding some more info about the app.
>>> > > >>
>>> > > >>> We are storing web page data in HBase.
>>> > > >>> Row key is Hashed URL, for random distribution, since we dont plan
>>> to
>>> > > do
>>> > > >> scan's..
>>> > > >>> We have LZOCompression Set on this column family.
>>> > > >>> We were noticing 1500 Reads, when reading the page content.
>>> > > >>> We have a column family, which stores just metadata of the page
>>> > "title"
>>> > > >> etc... When reading this the performance is whopping 12000 TPS.
>>> > > >>
>>> > > >> We though the issue could be because of N/w bandwidth used between
>>> > HBase
>>> > > >> and Clients. So we disable LZO Compression on Column Family and
>>> > started
>>> > > >> doing the compression of the raw page on the client and decompress
>>> it
>>> > > when
>>> > > >> readind (LZO).
>>> > > >>
>>> > > >>> With this my write performance jumped up from 2000 to 5000 at peak.
>>> > > >>> With this approach, the servers are crashing... Not sure , why only
>>> > > >> after
>>> > > >> turning of LZO... and doing the same from client.
>>> > > >>
>>> > > >>
>>> > > >>
>>> > > >> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com>
>>> wrote:
>>> > > >>
>>> > > >>> How much heap are you running on your RegionServers?
>>> > > >>>
>>> > > >>> 6GB of total RAM is on the low end.  For high throughput
>>> > applications,
>>> > > I
>>> > > >>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
>>> > > >>>
>>> > > >>>> -----Original Message-----
>>> > > >>>> From: charan kumar [mailto:charan.kumar@gmail.com]
>>> > > >>>> Sent: Thursday, February 03, 2011 11:47 AM
>>> > > >>>> To: user@hbase.apache.org
>>> > > >>>> Subject: Region Servers Crashing during Random Reads
>>> > > >>>>
>>> > > >>>> Hello,
>>> > > >>>>
>>> > > >>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2
>>> CPU,
>>> > 6
>>> > > >> GB
>>> > > >>>> RAM)
>>> > > >>>>
>>> > > >>>> I had 9 Region Servers crash (out of 30) in a span of 30 minutes
>>> > > during
>>> > > >> a
>>> > > >>> heavy
>>> > > >>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy to
>>> > me.
>>> > > >>>> I did all recommended configuration from the Hbase wiki... Any
>>> other
>>> > > >>>> suggestions?
>>> > > >>>>
>>> > > >>>>
>>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
>>> > > >>>> (promotion
>>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
>>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
>>> > > >>>>
>>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
>>> > > >>>> (promotion
>>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
>>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
>>> > > >>>>
>>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
>>> > > >>>> (promotion
>>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
>>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
>>> > > >>>>
>>> > > >>>>
>>> > > >>>> The following is the log entry in region Server
>>> > > >>>>
>>> > > >>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn:
>>> Client
>>> > > >>>> session timed out, have not heard from server in 47172ms for
>>> > sessionid
>>> > > >>>> 0x12db9f722421ce3, closing socket connection and attempting
>>> > reconnect
>>> > > >>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn:
>>> Client
>>> > > >>>> session timed out, have not heard from server in 48159ms for
>>> > sessionid
>>> > > >>>> 0x22db9f722501d93, closing socket connection and attempting
>>> > reconnect
>>> > > >>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn:
>>> > Opening
>>> > > >>>> socket connection to server XXXXXXXXXXXXXXXX
>>> > > >>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn:
>>> Socket
>>> > > >>>> connection established to XXXXXXXXX, initiating session
>>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
>>> > Opening
>>> > > >>>> socket connection to server XXXXXXXXXXXXXXX
>>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
>>> Socket
>>> > > >>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating
>>> session
>>> > > >>>> 2011-02-03 10:37:44,767 DEBUG
>>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
>>> > > >> eviction
>>> > > >>>> started; Attempting to free 81.93 MB of total=696.25 MB
>>> > > >>>> 2011-02-03 10:37:44,784 DEBUG
>>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
>>> > > >> eviction
>>> > > >>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
>>> > > >>>> multi=309.77 MB, memory=0 KB
>>> > > >>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn:
>>> Unable
>>> > > to
>>> > > >>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has
>>> > expired,
>>> > > >>>> closing socket connection
>>> > > >>>> 2011-02-03 10:37:45,206 INFO
>>> > > >>>>
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>>> > > >>>> entation:
>>> > > >>>> This client just lost it's session with ZooKeeper, trying to
>>> > > reconnect.
>>> > > >>>> 2011-02-03 10:37:45,453 INFO
>>> > > >>>>
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>>> > > >>>> entation:
>>> > > >>>> Trying to reconnect to zookeeper
>>> > > >>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn:
>>> Unable
>>> > > to
>>> > > >>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has
>>> > expired,
>>> > > >>>> closing socket connection
>>> > > >>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
>>> > > >>>> 0x22db9f722501d93
>>> > > >>>> received expired from ZooKeeper, aborting
>>> > > >>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
>>> > > >>>> KeeperErrorCode = Session expired
>>> > > >>>>        at
>>> > > >>>>
>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
>>> > > >>>> ZooKeeperWatcher.java:328)
>>> > > >>>>        at
>>> > > >>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
>>> > > >>>> erWatcher.java:246)
>>> > > >>>>        at
>>> > > >>>>
>>> > org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
>>> > > >>>> va:530)
>>> > > >>>>        at
>>> > > >>>>
>>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
>>> > > >>>> handled exception: org.apache.hadoop.hbase.YouAreDeadException:
>>> > Server
>>> > > >>>> REPORT rejected; currently processing
>>> > XXXXXXXXXXXX,60020,1296684296172
>>> > > >>>> as dead server
>>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException:
>>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT
>>> rejected;
>>> > > >>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead
>>> server
>>> > > >>>>        at
>>> > > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
>>> > > >>>> Method)
>>> > > >>>>        at
>>> > > >>>>
>>> > > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
>>> > > >>>> AccessorImpl.java:39)
>>> > > >>>>        at
>>> > > >>>>
>>> > > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
>>> > > >>>> structorAccessorImpl.java:27)
>>> > > >>>>        at
>>> > > >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>>> > > >>>>        at
>>> > > >>>>
>>> > org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
>>> > > >>>> ption.java:96)
>>> > > >>>>        at
>>> > > >>>> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
>>> > > >>>> Exception.java:80)
>>> > > >>>>        at
>>> > > >>>>
>>> > org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
>>> > > >>>> ort(HRegionServer.java:729)
>>> > > >>>>        at
>>> > > >>>>
>>> > org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
>>> > > >>>> ava:586)
>>> > > >>>>        at java.lang.Thread.run(Thread.java:619)
>>> > > >>>>
>>> > > >>>>
>>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
>>> > > >>>> (promotion
>>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
>>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
>>> > > >>>>
>>> > > >>>>
>>> > > >>>>
>>> > > >>>> Thanks,
>>> > > >>>> Charan
>>> > > >>>
>>> > > >>
>>> > > >
>>> > > >
>>> > > >
>>> > > > --
>>> > > > Todd Lipcon
>>> > > > Software Engineer, Cloudera
>>> > >
>>> >
>>>
>>>
>>>
>>> --
>>> Todd Lipcon
>>> Software Engineer, Cloudera
>>>
>>
>

Re: Region Servers Crashing during Random Reads

Posted by Stack <st...@duboce.net>.
Yeah, our wiki page seems way off to me.  I can update it.  Rather
than hardcoding absolute new gen size Todd, how about using
-XX:NewRatio=3 say; i.e. 1/4 of heap is new gen (maybe it should be
1/3rd!).  Does UseParNewGC do anything?  I seem to see the 'parallel'
rescans whether its on or off (This says its off by default,
http://www.md.pp.ru/~eu/jdk6options.html#UseParNewGC, but I trust my
eyes and this more
http://blogs.sun.com/jonthecollector/category/Java).  70% for
initiating fraction seems conservative (but I know what you are going
to say and yes, you are right we should be conservative....).  We
should tag on '-XX:+UseCMSInitiatingOccupancyOnly' too?

If you are good w/ above changes (I can leave UseParNewGC in the mix),
I'll make the changes to the wiki.

Good stuff,
St.Ack


On Thu, Feb 3, 2011 at 10:26 PM, charan kumar <ch...@gmail.com> wrote:
> Here you go..
>
> HBase Performance tuning page
> http://wiki.apache.org/hadoop/Hbase/FAQ#A7refers to the following
> hadoop URL.
>
> http://wiki.apache.org/hadoop/PerformanceTuning
>
> Thanks,
> Charan
>
>
> On Thu, Feb 3, 2011 at 10:22 PM, Todd Lipcon <to...@cloudera.com> wrote:
>
>> Does the wiki really recommend that? Got a link handy?
>>
>> On Thu, Feb 3, 2011 at 10:20 PM, charan kumar <charan.kumar@gmail.com
>> >wrote:
>>
>> > Todd,
>> >
>> >  That did the trick.  I think the wiki should be updated as well, no
>> point
>> > in recommending ParNew 6M or is it?
>> >
>> > Thanks,
>> > Charan.
>> >
>> > On Thu, Feb 3, 2011 at 2:06 PM, Charan K <ch...@gmail.com> wrote:
>> >
>> > > Thanks Todd.. I will try it out ..
>> > >
>> > >
>> > > On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:
>> > >
>> > > > Hi Charan,
>> > > >
>> > > > Your GC settings are way off - 6m newsize will promote way too much
>> to
>> > > the
>> > > > oldgen.
>> > > >
>> > > > Try this:
>> > > >
>> > > > -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
>> > > > -XX:CMSInitiatingOccupancyFraction=70
>> > > >
>> > > > -Todd
>> > > >
>> > > > On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <
>> charan.kumar@gmail.com
>> > > >wrote:
>> > > >
>> > > >> HI Jonathan,
>> > > >>
>> > > >> Thanks for you quick reply..
>> > > >>
>> > > >> Heap is set to 4G.
>> > > >>
>> > > >> Following are the JVM opts.
>> > > >> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
>> > > >> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
>> > > >> -XX:MaxNewSize=6m"
>> > > >>
>> > > >> Are there any other options apart from increasing the RAM?
>> > > >>
>> > > >> I am adding some more info about the app.
>> > > >>
>> > > >>> We are storing web page data in HBase.
>> > > >>> Row key is Hashed URL, for random distribution, since we dont plan
>> to
>> > > do
>> > > >> scan's..
>> > > >>> We have LZOCompression Set on this column family.
>> > > >>> We were noticing 1500 Reads, when reading the page content.
>> > > >>> We have a column family, which stores just metadata of the page
>> > "title"
>> > > >> etc... When reading this the performance is whopping 12000 TPS.
>> > > >>
>> > > >> We though the issue could be because of N/w bandwidth used between
>> > HBase
>> > > >> and Clients. So we disable LZO Compression on Column Family and
>> > started
>> > > >> doing the compression of the raw page on the client and decompress
>> it
>> > > when
>> > > >> readind (LZO).
>> > > >>
>> > > >>> With this my write performance jumped up from 2000 to 5000 at peak.
>> > > >>> With this approach, the servers are crashing... Not sure , why only
>> > > >> after
>> > > >> turning of LZO... and doing the same from client.
>> > > >>
>> > > >>
>> > > >>
>> > > >> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com>
>> wrote:
>> > > >>
>> > > >>> How much heap are you running on your RegionServers?
>> > > >>>
>> > > >>> 6GB of total RAM is on the low end.  For high throughput
>> > applications,
>> > > I
>> > > >>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
>> > > >>>
>> > > >>>> -----Original Message-----
>> > > >>>> From: charan kumar [mailto:charan.kumar@gmail.com]
>> > > >>>> Sent: Thursday, February 03, 2011 11:47 AM
>> > > >>>> To: user@hbase.apache.org
>> > > >>>> Subject: Region Servers Crashing during Random Reads
>> > > >>>>
>> > > >>>> Hello,
>> > > >>>>
>> > > >>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2
>> CPU,
>> > 6
>> > > >> GB
>> > > >>>> RAM)
>> > > >>>>
>> > > >>>> I had 9 Region Servers crash (out of 30) in a span of 30 minutes
>> > > during
>> > > >> a
>> > > >>> heavy
>> > > >>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy to
>> > me.
>> > > >>>> I did all recommended configuration from the Hbase wiki... Any
>> other
>> > > >>>> suggestions?
>> > > >>>>
>> > > >>>>
>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
>> > > >>>> (promotion
>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
>> > > >>>>
>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
>> > > >>>> (promotion
>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
>> > > >>>>
>> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
>> > > >>>> (promotion
>> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
>> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
>> > > >>>>
>> > > >>>>
>> > > >>>> The following is the log entry in region Server
>> > > >>>>
>> > > >>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn:
>> Client
>> > > >>>> session timed out, have not heard from server in 47172ms for
>> > sessionid
>> > > >>>> 0x12db9f722421ce3, closing socket connection and attempting
>> > reconnect
>> > > >>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn:
>> Client
>> > > >>>> session timed out, have not heard from server in 48159ms for
>> > sessionid
>> > > >>>> 0x22db9f722501d93, closing socket connection and attempting
>> > reconnect
>> > > >>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn:
>> > Opening
>> > > >>>> socket connection to server XXXXXXXXXXXXXXXX
>> > > >>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn:
>> Socket
>> > > >>>> connection established to XXXXXXXXX, initiating session
>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
>> > Opening
>> > > >>>> socket connection to server XXXXXXXXXXXXXXX
>> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
>> Socket
>> > > >>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating
>> session
>> > > >>>> 2011-02-03 10:37:44,767 DEBUG
>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
>> > > >> eviction
>> > > >>>> started; Attempting to free 81.93 MB of total=696.25 MB
>> > > >>>> 2011-02-03 10:37:44,784 DEBUG
>> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
>> > > >> eviction
>> > > >>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
>> > > >>>> multi=309.77 MB, memory=0 KB
>> > > >>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn:
>> Unable
>> > > to
>> > > >>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has
>> > expired,
>> > > >>>> closing socket connection
>> > > >>>> 2011-02-03 10:37:45,206 INFO
>> > > >>>>
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>> > > >>>> entation:
>> > > >>>> This client just lost it's session with ZooKeeper, trying to
>> > > reconnect.
>> > > >>>> 2011-02-03 10:37:45,453 INFO
>> > > >>>>
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>> > > >>>> entation:
>> > > >>>> Trying to reconnect to zookeeper
>> > > >>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn:
>> Unable
>> > > to
>> > > >>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has
>> > expired,
>> > > >>>> closing socket connection
>> > > >>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
>> > > >>>> 0x22db9f722501d93
>> > > >>>> received expired from ZooKeeper, aborting
>> > > >>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
>> > > >>>> KeeperErrorCode = Session expired
>> > > >>>>        at
>> > > >>>>
>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
>> > > >>>> ZooKeeperWatcher.java:328)
>> > > >>>>        at
>> > > >>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
>> > > >>>> erWatcher.java:246)
>> > > >>>>        at
>> > > >>>>
>> > org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
>> > > >>>> va:530)
>> > > >>>>        at
>> > > >>>>
>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
>> > > >>>> handled exception: org.apache.hadoop.hbase.YouAreDeadException:
>> > Server
>> > > >>>> REPORT rejected; currently processing
>> > XXXXXXXXXXXX,60020,1296684296172
>> > > >>>> as dead server
>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException:
>> > > >>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT
>> rejected;
>> > > >>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead
>> server
>> > > >>>>        at
>> > > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
>> > > >>>> Method)
>> > > >>>>        at
>> > > >>>>
>> > > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
>> > > >>>> AccessorImpl.java:39)
>> > > >>>>        at
>> > > >>>>
>> > > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
>> > > >>>> structorAccessorImpl.java:27)
>> > > >>>>        at
>> > > >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>> > > >>>>        at
>> > > >>>>
>> > org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
>> > > >>>> ption.java:96)
>> > > >>>>        at
>> > > >>>> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
>> > > >>>> Exception.java:80)
>> > > >>>>        at
>> > > >>>>
>> > org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
>> > > >>>> ort(HRegionServer.java:729)
>> > > >>>>        at
>> > > >>>>
>> > org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
>> > > >>>> ava:586)
>> > > >>>>        at java.lang.Thread.run(Thread.java:619)
>> > > >>>>
>> > > >>>>
>> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
>> > > >>>> (promotion
>> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
>> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
>> > > >>>>
>> > > >>>>
>> > > >>>>
>> > > >>>> Thanks,
>> > > >>>> Charan
>> > > >>>
>> > > >>
>> > > >
>> > > >
>> > > >
>> > > > --
>> > > > Todd Lipcon
>> > > > Software Engineer, Cloudera
>> > >
>> >
>>
>>
>>
>> --
>> Todd Lipcon
>> Software Engineer, Cloudera
>>
>

Re: Region Servers Crashing during Random Reads

Posted by charan kumar <ch...@gmail.com>.
Here you go..

HBase Performance tuning page
http://wiki.apache.org/hadoop/Hbase/FAQ#A7refers to the following
hadoop URL.

http://wiki.apache.org/hadoop/PerformanceTuning

Thanks,
Charan


On Thu, Feb 3, 2011 at 10:22 PM, Todd Lipcon <to...@cloudera.com> wrote:

> Does the wiki really recommend that? Got a link handy?
>
> On Thu, Feb 3, 2011 at 10:20 PM, charan kumar <charan.kumar@gmail.com
> >wrote:
>
> > Todd,
> >
> >  That did the trick.  I think the wiki should be updated as well, no
> point
> > in recommending ParNew 6M or is it?
> >
> > Thanks,
> > Charan.
> >
> > On Thu, Feb 3, 2011 at 2:06 PM, Charan K <ch...@gmail.com> wrote:
> >
> > > Thanks Todd.. I will try it out ..
> > >
> > >
> > > On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:
> > >
> > > > Hi Charan,
> > > >
> > > > Your GC settings are way off - 6m newsize will promote way too much
> to
> > > the
> > > > oldgen.
> > > >
> > > > Try this:
> > > >
> > > > -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
> > > > -XX:CMSInitiatingOccupancyFraction=70
> > > >
> > > > -Todd
> > > >
> > > > On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <
> charan.kumar@gmail.com
> > > >wrote:
> > > >
> > > >> HI Jonathan,
> > > >>
> > > >> Thanks for you quick reply..
> > > >>
> > > >> Heap is set to 4G.
> > > >>
> > > >> Following are the JVM opts.
> > > >> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
> > > >> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
> > > >> -XX:MaxNewSize=6m"
> > > >>
> > > >> Are there any other options apart from increasing the RAM?
> > > >>
> > > >> I am adding some more info about the app.
> > > >>
> > > >>> We are storing web page data in HBase.
> > > >>> Row key is Hashed URL, for random distribution, since we dont plan
> to
> > > do
> > > >> scan's..
> > > >>> We have LZOCompression Set on this column family.
> > > >>> We were noticing 1500 Reads, when reading the page content.
> > > >>> We have a column family, which stores just metadata of the page
> > "title"
> > > >> etc... When reading this the performance is whopping 12000 TPS.
> > > >>
> > > >> We though the issue could be because of N/w bandwidth used between
> > HBase
> > > >> and Clients. So we disable LZO Compression on Column Family and
> > started
> > > >> doing the compression of the raw page on the client and decompress
> it
> > > when
> > > >> readind (LZO).
> > > >>
> > > >>> With this my write performance jumped up from 2000 to 5000 at peak.
> > > >>> With this approach, the servers are crashing... Not sure , why only
> > > >> after
> > > >> turning of LZO... and doing the same from client.
> > > >>
> > > >>
> > > >>
> > > >> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com>
> wrote:
> > > >>
> > > >>> How much heap are you running on your RegionServers?
> > > >>>
> > > >>> 6GB of total RAM is on the low end.  For high throughput
> > applications,
> > > I
> > > >>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
> > > >>>
> > > >>>> -----Original Message-----
> > > >>>> From: charan kumar [mailto:charan.kumar@gmail.com]
> > > >>>> Sent: Thursday, February 03, 2011 11:47 AM
> > > >>>> To: user@hbase.apache.org
> > > >>>> Subject: Region Servers Crashing during Random Reads
> > > >>>>
> > > >>>> Hello,
> > > >>>>
> > > >>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2
> CPU,
> > 6
> > > >> GB
> > > >>>> RAM)
> > > >>>>
> > > >>>> I had 9 Region Servers crash (out of 30) in a span of 30 minutes
> > > during
> > > >> a
> > > >>> heavy
> > > >>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy to
> > me.
> > > >>>> I did all recommended configuration from the Hbase wiki... Any
> other
> > > >>>> suggestions?
> > > >>>>
> > > >>>>
> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > > >>>> (promotion
> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> > > >>>>
> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > > >>>> (promotion
> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> > > >>>>
> > > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > > >>>> (promotion
> > > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> > > >>>>
> > > >>>>
> > > >>>> The following is the log entry in region Server
> > > >>>>
> > > >>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn:
> Client
> > > >>>> session timed out, have not heard from server in 47172ms for
> > sessionid
> > > >>>> 0x12db9f722421ce3, closing socket connection and attempting
> > reconnect
> > > >>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn:
> Client
> > > >>>> session timed out, have not heard from server in 48159ms for
> > sessionid
> > > >>>> 0x22db9f722501d93, closing socket connection and attempting
> > reconnect
> > > >>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn:
> > Opening
> > > >>>> socket connection to server XXXXXXXXXXXXXXXX
> > > >>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn:
> Socket
> > > >>>> connection established to XXXXXXXXX, initiating session
> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
> > Opening
> > > >>>> socket connection to server XXXXXXXXXXXXXXX
> > > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
> Socket
> > > >>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating
> session
> > > >>>> 2011-02-03 10:37:44,767 DEBUG
> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> > > >> eviction
> > > >>>> started; Attempting to free 81.93 MB of total=696.25 MB
> > > >>>> 2011-02-03 10:37:44,784 DEBUG
> > > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> > > >> eviction
> > > >>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
> > > >>>> multi=309.77 MB, memory=0 KB
> > > >>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn:
> Unable
> > > to
> > > >>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has
> > expired,
> > > >>>> closing socket connection
> > > >>>> 2011-02-03 10:37:45,206 INFO
> > > >>>>
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > > >>>> entation:
> > > >>>> This client just lost it's session with ZooKeeper, trying to
> > > reconnect.
> > > >>>> 2011-02-03 10:37:45,453 INFO
> > > >>>>
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > > >>>> entation:
> > > >>>> Trying to reconnect to zookeeper
> > > >>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn:
> Unable
> > > to
> > > >>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has
> > expired,
> > > >>>> closing socket connection
> > > >>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
> > > >>>> 0x22db9f722501d93
> > > >>>> received expired from ZooKeeper, aborting
> > > >>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
> > > >>>> KeeperErrorCode = Session expired
> > > >>>>        at
> > > >>>>
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
> > > >>>> ZooKeeperWatcher.java:328)
> > > >>>>        at
> > > >>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
> > > >>>> erWatcher.java:246)
> > > >>>>        at
> > > >>>>
> > org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
> > > >>>> va:530)
> > > >>>>        at
> > > >>>>
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
> > > >>>> handled exception: org.apache.hadoop.hbase.YouAreDeadException:
> > Server
> > > >>>> REPORT rejected; currently processing
> > XXXXXXXXXXXX,60020,1296684296172
> > > >>>> as dead server
> > > >>>> org.apache.hadoop.hbase.YouAreDeadException:
> > > >>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT
> rejected;
> > > >>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead
> server
> > > >>>>        at
> > > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> > > >>>> Method)
> > > >>>>        at
> > > >>>>
> > > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
> > > >>>> AccessorImpl.java:39)
> > > >>>>        at
> > > >>>>
> > > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
> > > >>>> structorAccessorImpl.java:27)
> > > >>>>        at
> > > >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
> > > >>>>        at
> > > >>>>
> > org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
> > > >>>> ption.java:96)
> > > >>>>        at
> > > >>>> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
> > > >>>> Exception.java:80)
> > > >>>>        at
> > > >>>>
> > org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
> > > >>>> ort(HRegionServer.java:729)
> > > >>>>        at
> > > >>>>
> > org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
> > > >>>> ava:586)
> > > >>>>        at java.lang.Thread.run(Thread.java:619)
> > > >>>>
> > > >>>>
> > > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > > >>>> (promotion
> > > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> Thanks,
> > > >>>> Charan
> > > >>>
> > > >>
> > > >
> > > >
> > > >
> > > > --
> > > > Todd Lipcon
> > > > Software Engineer, Cloudera
> > >
> >
>
>
>
> --
> Todd Lipcon
> Software Engineer, Cloudera
>

Re: Region Servers Crashing during Random Reads

Posted by Todd Lipcon <to...@cloudera.com>.
Does the wiki really recommend that? Got a link handy?

On Thu, Feb 3, 2011 at 10:20 PM, charan kumar <ch...@gmail.com>wrote:

> Todd,
>
>  That did the trick.  I think the wiki should be updated as well, no point
> in recommending ParNew 6M or is it?
>
> Thanks,
> Charan.
>
> On Thu, Feb 3, 2011 at 2:06 PM, Charan K <ch...@gmail.com> wrote:
>
> > Thanks Todd.. I will try it out ..
> >
> >
> > On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:
> >
> > > Hi Charan,
> > >
> > > Your GC settings are way off - 6m newsize will promote way too much to
> > the
> > > oldgen.
> > >
> > > Try this:
> > >
> > > -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
> > > -XX:CMSInitiatingOccupancyFraction=70
> > >
> > > -Todd
> > >
> > > On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <charan.kumar@gmail.com
> > >wrote:
> > >
> > >> HI Jonathan,
> > >>
> > >> Thanks for you quick reply..
> > >>
> > >> Heap is set to 4G.
> > >>
> > >> Following are the JVM opts.
> > >> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
> > >> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
> > >> -XX:MaxNewSize=6m"
> > >>
> > >> Are there any other options apart from increasing the RAM?
> > >>
> > >> I am adding some more info about the app.
> > >>
> > >>> We are storing web page data in HBase.
> > >>> Row key is Hashed URL, for random distribution, since we dont plan to
> > do
> > >> scan's..
> > >>> We have LZOCompression Set on this column family.
> > >>> We were noticing 1500 Reads, when reading the page content.
> > >>> We have a column family, which stores just metadata of the page
> "title"
> > >> etc... When reading this the performance is whopping 12000 TPS.
> > >>
> > >> We though the issue could be because of N/w bandwidth used between
> HBase
> > >> and Clients. So we disable LZO Compression on Column Family and
> started
> > >> doing the compression of the raw page on the client and decompress it
> > when
> > >> readind (LZO).
> > >>
> > >>> With this my write performance jumped up from 2000 to 5000 at peak.
> > >>> With this approach, the servers are crashing... Not sure , why only
> > >> after
> > >> turning of LZO... and doing the same from client.
> > >>
> > >>
> > >>
> > >> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com> wrote:
> > >>
> > >>> How much heap are you running on your RegionServers?
> > >>>
> > >>> 6GB of total RAM is on the low end.  For high throughput
> applications,
> > I
> > >>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
> > >>>
> > >>>> -----Original Message-----
> > >>>> From: charan kumar [mailto:charan.kumar@gmail.com]
> > >>>> Sent: Thursday, February 03, 2011 11:47 AM
> > >>>> To: user@hbase.apache.org
> > >>>> Subject: Region Servers Crashing during Random Reads
> > >>>>
> > >>>> Hello,
> > >>>>
> > >>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2 CPU,
> 6
> > >> GB
> > >>>> RAM)
> > >>>>
> > >>>> I had 9 Region Servers crash (out of 30) in a span of 30 minutes
> > during
> > >> a
> > >>> heavy
> > >>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy to
> me.
> > >>>> I did all recommended configuration from the Hbase wiki... Any other
> > >>>> suggestions?
> > >>>>
> > >>>>
> > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > >>>> (promotion
> > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> > >>>>
> > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > >>>> (promotion
> > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> > >>>>
> > >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > >>>> (promotion
> > >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> > >>>>
> > >>>>
> > >>>> The following is the log entry in region Server
> > >>>>
> > >>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn: Client
> > >>>> session timed out, have not heard from server in 47172ms for
> sessionid
> > >>>> 0x12db9f722421ce3, closing socket connection and attempting
> reconnect
> > >>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn: Client
> > >>>> session timed out, have not heard from server in 48159ms for
> sessionid
> > >>>> 0x22db9f722501d93, closing socket connection and attempting
> reconnect
> > >>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn:
> Opening
> > >>>> socket connection to server XXXXXXXXXXXXXXXX
> > >>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn: Socket
> > >>>> connection established to XXXXXXXXX, initiating session
> > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn:
> Opening
> > >>>> socket connection to server XXXXXXXXXXXXXXX
> > >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Socket
> > >>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating session
> > >>>> 2011-02-03 10:37:44,767 DEBUG
> > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> > >> eviction
> > >>>> started; Attempting to free 81.93 MB of total=696.25 MB
> > >>>> 2011-02-03 10:37:44,784 DEBUG
> > >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> > >> eviction
> > >>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
> > >>>> multi=309.77 MB, memory=0 KB
> > >>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn: Unable
> > to
> > >>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has
> expired,
> > >>>> closing socket connection
> > >>>> 2011-02-03 10:37:45,206 INFO
> > >>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > >>>> entation:
> > >>>> This client just lost it's session with ZooKeeper, trying to
> > reconnect.
> > >>>> 2011-02-03 10:37:45,453 INFO
> > >>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > >>>> entation:
> > >>>> Trying to reconnect to zookeeper
> > >>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn: Unable
> > to
> > >>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has
> expired,
> > >>>> closing socket connection
> > >>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
> > >>>> 0x22db9f722501d93
> > >>>> received expired from ZooKeeper, aborting
> > >>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
> > >>>> KeeperErrorCode = Session expired
> > >>>>        at
> > >>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
> > >>>> ZooKeeperWatcher.java:328)
> > >>>>        at
> > >>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
> > >>>> erWatcher.java:246)
> > >>>>        at
> > >>>>
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
> > >>>> va:530)
> > >>>>        at
> > >>>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
> > >>>> handled exception: org.apache.hadoop.hbase.YouAreDeadException:
> Server
> > >>>> REPORT rejected; currently processing
> XXXXXXXXXXXX,60020,1296684296172
> > >>>> as dead server
> > >>>> org.apache.hadoop.hbase.YouAreDeadException:
> > >>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT rejected;
> > >>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead server
> > >>>>        at
> > >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> > >>>> Method)
> > >>>>        at
> > >>>>
> > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
> > >>>> AccessorImpl.java:39)
> > >>>>        at
> > >>>>
> > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
> > >>>> structorAccessorImpl.java:27)
> > >>>>        at
> > >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
> > >>>>        at
> > >>>>
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
> > >>>> ption.java:96)
> > >>>>        at
> > >>>> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
> > >>>> Exception.java:80)
> > >>>>        at
> > >>>>
> org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
> > >>>> ort(HRegionServer.java:729)
> > >>>>        at
> > >>>>
> org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
> > >>>> ava:586)
> > >>>>        at java.lang.Thread.run(Thread.java:619)
> > >>>>
> > >>>>
> > >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > >>>> (promotion
> > >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> > >>>>
> > >>>>
> > >>>>
> > >>>> Thanks,
> > >>>> Charan
> > >>>
> > >>
> > >
> > >
> > >
> > > --
> > > Todd Lipcon
> > > Software Engineer, Cloudera
> >
>



-- 
Todd Lipcon
Software Engineer, Cloudera

Re: Region Servers Crashing during Random Reads

Posted by charan kumar <ch...@gmail.com>.
Todd,

  That did the trick.  I think the wiki should be updated as well, no point
in recommending ParNew 6M or is it?

Thanks,
Charan.

On Thu, Feb 3, 2011 at 2:06 PM, Charan K <ch...@gmail.com> wrote:

> Thanks Todd.. I will try it out ..
>
>
> On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:
>
> > Hi Charan,
> >
> > Your GC settings are way off - 6m newsize will promote way too much to
> the
> > oldgen.
> >
> > Try this:
> >
> > -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
> > -XX:CMSInitiatingOccupancyFraction=70
> >
> > -Todd
> >
> > On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <charan.kumar@gmail.com
> >wrote:
> >
> >> HI Jonathan,
> >>
> >> Thanks for you quick reply..
> >>
> >> Heap is set to 4G.
> >>
> >> Following are the JVM opts.
> >> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
> >> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
> >> -XX:MaxNewSize=6m"
> >>
> >> Are there any other options apart from increasing the RAM?
> >>
> >> I am adding some more info about the app.
> >>
> >>> We are storing web page data in HBase.
> >>> Row key is Hashed URL, for random distribution, since we dont plan to
> do
> >> scan's..
> >>> We have LZOCompression Set on this column family.
> >>> We were noticing 1500 Reads, when reading the page content.
> >>> We have a column family, which stores just metadata of the page "title"
> >> etc... When reading this the performance is whopping 12000 TPS.
> >>
> >> We though the issue could be because of N/w bandwidth used between HBase
> >> and Clients. So we disable LZO Compression on Column Family and started
> >> doing the compression of the raw page on the client and decompress it
> when
> >> readind (LZO).
> >>
> >>> With this my write performance jumped up from 2000 to 5000 at peak.
> >>> With this approach, the servers are crashing... Not sure , why only
> >> after
> >> turning of LZO... and doing the same from client.
> >>
> >>
> >>
> >> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com> wrote:
> >>
> >>> How much heap are you running on your RegionServers?
> >>>
> >>> 6GB of total RAM is on the low end.  For high throughput applications,
> I
> >>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
> >>>
> >>>> -----Original Message-----
> >>>> From: charan kumar [mailto:charan.kumar@gmail.com]
> >>>> Sent: Thursday, February 03, 2011 11:47 AM
> >>>> To: user@hbase.apache.org
> >>>> Subject: Region Servers Crashing during Random Reads
> >>>>
> >>>> Hello,
> >>>>
> >>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2 CPU, 6
> >> GB
> >>>> RAM)
> >>>>
> >>>> I had 9 Region Servers crash (out of 30) in a span of 30 minutes
> during
> >> a
> >>> heavy
> >>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy to me.
> >>>> I did all recommended configuration from the Hbase wiki... Any other
> >>>> suggestions?
> >>>>
> >>>>
> >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> >>>> (promotion
> >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> >>>>
> >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> >>>> (promotion
> >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> >>>>
> >>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> >>>> (promotion
> >>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> >>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> >>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> >>>>
> >>>>
> >>>> The following is the log entry in region Server
> >>>>
> >>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn: Client
> >>>> session timed out, have not heard from server in 47172ms for sessionid
> >>>> 0x12db9f722421ce3, closing socket connection and attempting reconnect
> >>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn: Client
> >>>> session timed out, have not heard from server in 48159ms for sessionid
> >>>> 0x22db9f722501d93, closing socket connection and attempting reconnect
> >>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn: Opening
> >>>> socket connection to server XXXXXXXXXXXXXXXX
> >>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn: Socket
> >>>> connection established to XXXXXXXXX, initiating session
> >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Opening
> >>>> socket connection to server XXXXXXXXXXXXXXX
> >>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Socket
> >>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating session
> >>>> 2011-02-03 10:37:44,767 DEBUG
> >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> >> eviction
> >>>> started; Attempting to free 81.93 MB of total=696.25 MB
> >>>> 2011-02-03 10:37:44,784 DEBUG
> >>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> >> eviction
> >>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
> >>>> multi=309.77 MB, memory=0 KB
> >>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn: Unable
> to
> >>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has expired,
> >>>> closing socket connection
> >>>> 2011-02-03 10:37:45,206 INFO
> >>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> >>>> entation:
> >>>> This client just lost it's session with ZooKeeper, trying to
> reconnect.
> >>>> 2011-02-03 10:37:45,453 INFO
> >>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> >>>> entation:
> >>>> Trying to reconnect to zookeeper
> >>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn: Unable
> to
> >>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has expired,
> >>>> closing socket connection
> >>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
> >>>> 0x22db9f722501d93
> >>>> received expired from ZooKeeper, aborting
> >>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
> >>>> KeeperErrorCode = Session expired
> >>>>        at
> >>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
> >>>> ZooKeeperWatcher.java:328)
> >>>>        at
> >>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
> >>>> erWatcher.java:246)
> >>>>        at
> >>>> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
> >>>> va:530)
> >>>>        at
> >>>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
> >>>> handled exception: org.apache.hadoop.hbase.YouAreDeadException: Server
> >>>> REPORT rejected; currently processing XXXXXXXXXXXX,60020,1296684296172
> >>>> as dead server
> >>>> org.apache.hadoop.hbase.YouAreDeadException:
> >>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT rejected;
> >>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead server
> >>>>        at
> >> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> >>>> Method)
> >>>>        at
> >>>>
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
> >>>> AccessorImpl.java:39)
> >>>>        at
> >>>>
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
> >>>> structorAccessorImpl.java:27)
> >>>>        at
> >>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
> >>>>        at
> >>>> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
> >>>> ption.java:96)
> >>>>        at
> >>>> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
> >>>> Exception.java:80)
> >>>>        at
> >>>> org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
> >>>> ort(HRegionServer.java:729)
> >>>>        at
> >>>> org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
> >>>> ava:586)
> >>>>        at java.lang.Thread.run(Thread.java:619)
> >>>>
> >>>>
> >>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> >>>> (promotion
> >>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> >>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> >>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> >>>>
> >>>>
> >>>>
> >>>> Thanks,
> >>>> Charan
> >>>
> >>
> >
> >
> >
> > --
> > Todd Lipcon
> > Software Engineer, Cloudera
>

Re: Region Servers Crashing during Random Reads

Posted by Charan K <ch...@gmail.com>.
Thanks Todd.. I will try it out ..


On Feb 3, 2011, at 1:43 PM, Todd Lipcon <to...@cloudera.com> wrote:

> Hi Charan,
> 
> Your GC settings are way off - 6m newsize will promote way too much to the
> oldgen.
> 
> Try this:
> 
> -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
> -XX:CMSInitiatingOccupancyFraction=70
> 
> -Todd
> 
> On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <ch...@gmail.com>wrote:
> 
>> HI Jonathan,
>> 
>> Thanks for you quick reply..
>> 
>> Heap is set to 4G.
>> 
>> Following are the JVM opts.
>> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
>> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
>> -XX:MaxNewSize=6m"
>> 
>> Are there any other options apart from increasing the RAM?
>> 
>> I am adding some more info about the app.
>> 
>>> We are storing web page data in HBase.
>>> Row key is Hashed URL, for random distribution, since we dont plan to do
>> scan's..
>>> We have LZOCompression Set on this column family.
>>> We were noticing 1500 Reads, when reading the page content.
>>> We have a column family, which stores just metadata of the page "title"
>> etc... When reading this the performance is whopping 12000 TPS.
>> 
>> We though the issue could be because of N/w bandwidth used between HBase
>> and Clients. So we disable LZO Compression on Column Family and started
>> doing the compression of the raw page on the client and decompress it when
>> readind (LZO).
>> 
>>> With this my write performance jumped up from 2000 to 5000 at peak.
>>> With this approach, the servers are crashing... Not sure , why only
>> after
>> turning of LZO... and doing the same from client.
>> 
>> 
>> 
>> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com> wrote:
>> 
>>> How much heap are you running on your RegionServers?
>>> 
>>> 6GB of total RAM is on the low end.  For high throughput applications, I
>>> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
>>> 
>>>> -----Original Message-----
>>>> From: charan kumar [mailto:charan.kumar@gmail.com]
>>>> Sent: Thursday, February 03, 2011 11:47 AM
>>>> To: user@hbase.apache.org
>>>> Subject: Region Servers Crashing during Random Reads
>>>> 
>>>> Hello,
>>>> 
>>>> I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2 CPU, 6
>> GB
>>>> RAM)
>>>> 
>>>> I had 9 Region Servers crash (out of 30) in a span of 30 minutes during
>> a
>>> heavy
>>>> reads. It looks like a GC, ZooKeeper Connection Timeout thingy to me.
>>>> I did all recommended configuration from the Hbase wiki... Any other
>>>> suggestions?
>>>> 
>>>> 
>>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
>>>> (promotion
>>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
>>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
>>>> 
>>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
>>>> (promotion
>>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
>>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
>>>> 
>>>> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
>>>> (promotion
>>>> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
>>>> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
>>>> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
>>>> 
>>>> 
>>>> The following is the log entry in region Server
>>>> 
>>>> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn: Client
>>>> session timed out, have not heard from server in 47172ms for sessionid
>>>> 0x12db9f722421ce3, closing socket connection and attempting reconnect
>>>> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn: Client
>>>> session timed out, have not heard from server in 48159ms for sessionid
>>>> 0x22db9f722501d93, closing socket connection and attempting reconnect
>>>> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn: Opening
>>>> socket connection to server XXXXXXXXXXXXXXXX
>>>> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn: Socket
>>>> connection established to XXXXXXXXX, initiating session
>>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Opening
>>>> socket connection to server XXXXXXXXXXXXXXX
>>>> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Socket
>>>> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating session
>>>> 2011-02-03 10:37:44,767 DEBUG
>>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
>> eviction
>>>> started; Attempting to free 81.93 MB of total=696.25 MB
>>>> 2011-02-03 10:37:44,784 DEBUG
>>>> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
>> eviction
>>>> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
>>>> multi=309.77 MB, memory=0 KB
>>>> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn: Unable to
>>>> reconnect to ZooKeeper service, session 0x22db9f722501d93 has expired,
>>>> closing socket connection
>>>> 2011-02-03 10:37:45,206 INFO
>>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>>>> entation:
>>>> This client just lost it's session with ZooKeeper, trying to reconnect.
>>>> 2011-02-03 10:37:45,453 INFO
>>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
>>>> entation:
>>>> Trying to reconnect to zookeeper
>>>> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn: Unable to
>>>> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has expired,
>>>> closing socket connection
>>>> gionserver:60020-0x22db9f722501d93 regionserver:60020-
>>>> 0x22db9f722501d93
>>>> received expired from ZooKeeper, aborting
>>>> org.apache.zookeeper.KeeperException$SessionExpiredException:
>>>> KeeperErrorCode = Session expired
>>>>        at
>>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
>>>> ZooKeeperWatcher.java:328)
>>>>        at
>>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
>>>> erWatcher.java:246)
>>>>        at
>>>> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
>>>> va:530)
>>>>        at
>>>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
>>>> handled exception: org.apache.hadoop.hbase.YouAreDeadException: Server
>>>> REPORT rejected; currently processing XXXXXXXXXXXX,60020,1296684296172
>>>> as dead server
>>>> org.apache.hadoop.hbase.YouAreDeadException:
>>>> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT rejected;
>>>> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead server
>>>>        at
>> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
>>>> Method)
>>>>        at
>>>> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
>>>> AccessorImpl.java:39)
>>>>        at
>>>> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
>>>> structorAccessorImpl.java:27)
>>>>        at
>>> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>>>>        at
>>>> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
>>>> ption.java:96)
>>>>        at
>>>> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
>>>> Exception.java:80)
>>>>        at
>>>> org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
>>>> ort(HRegionServer.java:729)
>>>>        at
>>>> org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
>>>> ava:586)
>>>>        at java.lang.Thread.run(Thread.java:619)
>>>> 
>>>> 
>>>> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
>>>> (promotion
>>>> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
>>>> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
>>>> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
>>>> 
>>>> 
>>>> 
>>>> Thanks,
>>>> Charan
>>> 
>> 
> 
> 
> 
> -- 
> Todd Lipcon
> Software Engineer, Cloudera

Re: Region Servers Crashing during Random Reads

Posted by Todd Lipcon <to...@cloudera.com>.
Hi Charan,

Your GC settings are way off - 6m newsize will promote way too much to the
oldgen.

Try this:

-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -Xmn256m
-XX:CMSInitiatingOccupancyFraction=70

-Todd

On Thu, Feb 3, 2011 at 12:28 PM, charan kumar <ch...@gmail.com>wrote:

> HI Jonathan,
>
>  Thanks for you quick reply..
>
> Heap is set to 4G.
>
> Following are the JVM opts.
> export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
> -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
> -XX:MaxNewSize=6m"
>
> Are there any other options apart from increasing the RAM?
>
> I am adding some more info about the app.
>
>  > We are storing web page data in HBase.
>  > Row key is Hashed URL, for random distribution, since we dont plan to do
> scan's..
>  > We have LZOCompression Set on this column family.
>  > We were noticing 1500 Reads, when reading the page content.
>  > We have a column family, which stores just metadata of the page "title"
> etc... When reading this the performance is whopping 12000 TPS.
>
>  We though the issue could be because of N/w bandwidth used between HBase
> and Clients. So we disable LZO Compression on Column Family and started
> doing the compression of the raw page on the client and decompress it when
> readind (LZO).
>
>  > With this my write performance jumped up from 2000 to 5000 at peak.
>  > With this approach, the servers are crashing... Not sure , why only
> after
> turning of LZO... and doing the same from client.
>
>
>
> On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com> wrote:
>
> > How much heap are you running on your RegionServers?
> >
> > 6GB of total RAM is on the low end.  For high throughput applications, I
> > would recommend at least 6-8GB of heap (so 8+ GB of RAM).
> >
> > > -----Original Message-----
> > > From: charan kumar [mailto:charan.kumar@gmail.com]
> > > Sent: Thursday, February 03, 2011 11:47 AM
> > > To: user@hbase.apache.org
> > > Subject: Region Servers Crashing during Random Reads
> > >
> > > Hello,
> > >
> > >  I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2 CPU, 6
> GB
> > > RAM)
> > >
> > > I had 9 Region Servers crash (out of 30) in a span of 30 minutes during
> a
> > heavy
> > > reads. It looks like a GC, ZooKeeper Connection Timeout thingy to me.
> > > I did all recommended configuration from the Hbase wiki... Any other
> > > suggestions?
> > >
> > >
> > > 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > > (promotion
> > > failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > > [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > > 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> > >
> > > 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > > (promotion
> > > failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > > [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > > 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> > >
> > > 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > > (promotion
> > > failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > > [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > > 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> > >
> > >
> > > The following is the log entry in region Server
> > >
> > > 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn: Client
> > > session timed out, have not heard from server in 47172ms for sessionid
> > > 0x12db9f722421ce3, closing socket connection and attempting reconnect
> > > 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn: Client
> > > session timed out, have not heard from server in 48159ms for sessionid
> > > 0x22db9f722501d93, closing socket connection and attempting reconnect
> > > 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn: Opening
> > > socket connection to server XXXXXXXXXXXXXXXX
> > > 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn: Socket
> > > connection established to XXXXXXXXX, initiating session
> > > 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Opening
> > > socket connection to server XXXXXXXXXXXXXXX
> > > 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Socket
> > > connection established to XXXXXXXXXXXXXXXXXXXXX, initiating session
> > > 2011-02-03 10:37:44,767 DEBUG
> > > org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> eviction
> > > started; Attempting to free 81.93 MB of total=696.25 MB
> > > 2011-02-03 10:37:44,784 DEBUG
> > > org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU
> eviction
> > > completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
> > > multi=309.77 MB, memory=0 KB
> > > 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn: Unable to
> > > reconnect to ZooKeeper service, session 0x22db9f722501d93 has expired,
> > > closing socket connection
> > > 2011-02-03 10:37:45,206 INFO
> > > org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > > entation:
> > > This client just lost it's session with ZooKeeper, trying to reconnect.
> > > 2011-02-03 10:37:45,453 INFO
> > > org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > > entation:
> > > Trying to reconnect to zookeeper
> > > 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn: Unable to
> > > reconnect to ZooKeeper service, session 0x12db9f722421ce3 has expired,
> > > closing socket connection
> > > gionserver:60020-0x22db9f722501d93 regionserver:60020-
> > > 0x22db9f722501d93
> > > received expired from ZooKeeper, aborting
> > > org.apache.zookeeper.KeeperException$SessionExpiredException:
> > > KeeperErrorCode = Session expired
> > >         at
> > > org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
> > > ZooKeeperWatcher.java:328)
> > >         at
> > > org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
> > > erWatcher.java:246)
> > >         at
> > > org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
> > > va:530)
> > >         at
> > > org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
> > > handled exception: org.apache.hadoop.hbase.YouAreDeadException: Server
> > > REPORT rejected; currently processing XXXXXXXXXXXX,60020,1296684296172
> > > as dead server
> > > org.apache.hadoop.hbase.YouAreDeadException:
> > > org.apache.hadoop.hbase.YouAreDeadException: Server REPORT rejected;
> > > currently processing XXXXXXXXXXXX,60020,1296684296172 as dead server
> > >         at
> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> > > Method)
> > >         at
> > > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
> > > AccessorImpl.java:39)
> > >         at
> > > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
> > > structorAccessorImpl.java:27)
> > >         at
> > java.lang.reflect.Constructor.newInstance(Constructor.java:513)
> > >         at
> > > org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
> > > ption.java:96)
> > >         at
> > > org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
> > > Exception.java:80)
> > >         at
> > > org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
> > > ort(HRegionServer.java:729)
> > >         at
> > > org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
> > > ava:586)
> > >         at java.lang.Thread.run(Thread.java:619)
> > >
> > >
> > > 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > > (promotion
> > > failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > > [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > > 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> > >
> > >
> > >
> > > Thanks,
> > > Charan
> >
>



-- 
Todd Lipcon
Software Engineer, Cloudera

Re: Region Servers Crashing during Random Reads

Posted by charan kumar <ch...@gmail.com>.
HI Jonathan,

  Thanks for you quick reply..

Heap is set to 4G.

Following are the JVM opts.
export HBASE_OPTS="$HBASE_OPTS -XX:+HeapDumpOnOutOfMemoryError
-XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:NewSize=6m
-XX:MaxNewSize=6m"

Are there any other options apart from increasing the RAM?

I am adding some more info about the app.

 > We are storing web page data in HBase.
 > Row key is Hashed URL, for random distribution, since we dont plan to do
scan's..
 > We have LZOCompression Set on this column family.
 > We were noticing 1500 Reads, when reading the page content.
 > We have a column family, which stores just metadata of the page "title"
etc... When reading this the performance is whopping 12000 TPS.

  We though the issue could be because of N/w bandwidth used between HBase
and Clients. So we disable LZO Compression on Column Family and started
doing the compression of the raw page on the client and decompress it when
readind (LZO).

 > With this my write performance jumped up from 2000 to 5000 at peak.
 > With this approach, the servers are crashing... Not sure , why only after
turning of LZO... and doing the same from client.



On Thu, Feb 3, 2011 at 12:13 PM, Jonathan Gray <jg...@fb.com> wrote:

> How much heap are you running on your RegionServers?
>
> 6GB of total RAM is on the low end.  For high throughput applications, I
> would recommend at least 6-8GB of heap (so 8+ GB of RAM).
>
> > -----Original Message-----
> > From: charan kumar [mailto:charan.kumar@gmail.com]
> > Sent: Thursday, February 03, 2011 11:47 AM
> > To: user@hbase.apache.org
> > Subject: Region Servers Crashing during Random Reads
> >
> > Hello,
> >
> >  I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2 CPU, 6 GB
> > RAM)
> >
> > I had 9 Region Servers crash (out of 30) in a span of 30 minutes during a
> heavy
> > reads. It looks like a GC, ZooKeeper Connection Timeout thingy to me.
> > I did all recommended configuration from the Hbase wiki... Any other
> > suggestions?
> >
> >
> > 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > (promotion
> > failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> >
> > 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > (promotion
> > failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> >
> > 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> > (promotion
> > failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> > [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> > 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> >
> >
> > The following is the log entry in region Server
> >
> > 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn: Client
> > session timed out, have not heard from server in 47172ms for sessionid
> > 0x12db9f722421ce3, closing socket connection and attempting reconnect
> > 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn: Client
> > session timed out, have not heard from server in 48159ms for sessionid
> > 0x22db9f722501d93, closing socket connection and attempting reconnect
> > 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn: Opening
> > socket connection to server XXXXXXXXXXXXXXXX
> > 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn: Socket
> > connection established to XXXXXXXXX, initiating session
> > 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Opening
> > socket connection to server XXXXXXXXXXXXXXX
> > 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Socket
> > connection established to XXXXXXXXXXXXXXXXXXXXX, initiating session
> > 2011-02-03 10:37:44,767 DEBUG
> > org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU eviction
> > started; Attempting to free 81.93 MB of total=696.25 MB
> > 2011-02-03 10:37:44,784 DEBUG
> > org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU eviction
> > completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
> > multi=309.77 MB, memory=0 KB
> > 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn: Unable to
> > reconnect to ZooKeeper service, session 0x22db9f722501d93 has expired,
> > closing socket connection
> > 2011-02-03 10:37:45,206 INFO
> > org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > entation:
> > This client just lost it's session with ZooKeeper, trying to reconnect.
> > 2011-02-03 10:37:45,453 INFO
> > org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> > entation:
> > Trying to reconnect to zookeeper
> > 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn: Unable to
> > reconnect to ZooKeeper service, session 0x12db9f722421ce3 has expired,
> > closing socket connection
> > gionserver:60020-0x22db9f722501d93 regionserver:60020-
> > 0x22db9f722501d93
> > received expired from ZooKeeper, aborting
> > org.apache.zookeeper.KeeperException$SessionExpiredException:
> > KeeperErrorCode = Session expired
> >         at
> > org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
> > ZooKeeperWatcher.java:328)
> >         at
> > org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
> > erWatcher.java:246)
> >         at
> > org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
> > va:530)
> >         at
> > org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
> > handled exception: org.apache.hadoop.hbase.YouAreDeadException: Server
> > REPORT rejected; currently processing XXXXXXXXXXXX,60020,1296684296172
> > as dead server
> > org.apache.hadoop.hbase.YouAreDeadException:
> > org.apache.hadoop.hbase.YouAreDeadException: Server REPORT rejected;
> > currently processing XXXXXXXXXXXX,60020,1296684296172 as dead server
> >         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> > Method)
> >         at
> > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
> > AccessorImpl.java:39)
> >         at
> > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
> > structorAccessorImpl.java:27)
> >         at
> java.lang.reflect.Constructor.newInstance(Constructor.java:513)
> >         at
> > org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
> > ption.java:96)
> >         at
> > org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
> > Exception.java:80)
> >         at
> > org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
> > ort(HRegionServer.java:729)
> >         at
> > org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
> > ava:586)
> >         at java.lang.Thread.run(Thread.java:619)
> >
> >
> > 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> > (promotion
> > failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> > [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> > 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> >
> >
> >
> > Thanks,
> > Charan
>

RE: Region Servers Crashing during Random Reads

Posted by Jonathan Gray <jg...@fb.com>.
How much heap are you running on your RegionServers?

6GB of total RAM is on the low end.  For high throughput applications, I would recommend at least 6-8GB of heap (so 8+ GB of RAM).

> -----Original Message-----
> From: charan kumar [mailto:charan.kumar@gmail.com]
> Sent: Thursday, February 03, 2011 11:47 AM
> To: user@hbase.apache.org
> Subject: Region Servers Crashing during Random Reads
> 
> Hello,
> 
>  I am using hbase 0.90.0 with hadoop-append. h/w ( Dell 1950, 2 CPU, 6 GB
> RAM)
> 
> I had 9 Region Servers crash (out of 30) in a span of 30 minutes during a heavy
> reads. It looks like a GC, ZooKeeper Connection Timeout thingy to me.
> I did all recommended configuration from the Hbase wiki... Any other
> suggestions?
> 
> 
> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> (promotion
> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> 
> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> (promotion
> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> 
> 2011-02-03T09:43:07.890-0800: 70693.632: [GC 70693.632: [ParNew
> (promotion
> failed): 5555K->5540K(5568K), 0.0280950 secs]70693.660:
> [CMS2011-02-03T09:43:16.864-0800: 70702.606: [CMS-concurrent-mark:
> 12.549/69.323 secs] [Times: user=11.90 sys=1.26, real=69.31 secs]
> 
> 
> The following is the log entry in region Server
> 
> 2011-02-03 10:37:43,946 INFO org.apache.zookeeper.ClientCnxn: Client
> session timed out, have not heard from server in 47172ms for sessionid
> 0x12db9f722421ce3, closing socket connection and attempting reconnect
> 2011-02-03 10:37:43,947 INFO org.apache.zookeeper.ClientCnxn: Client
> session timed out, have not heard from server in 48159ms for sessionid
> 0x22db9f722501d93, closing socket connection and attempting reconnect
> 2011-02-03 10:37:44,401 INFO org.apache.zookeeper.ClientCnxn: Opening
> socket connection to server XXXXXXXXXXXXXXXX
> 2011-02-03 10:37:44,402 INFO org.apache.zookeeper.ClientCnxn: Socket
> connection established to XXXXXXXXX, initiating session
> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Opening
> socket connection to server XXXXXXXXXXXXXXX
> 2011-02-03 10:37:44,709 INFO org.apache.zookeeper.ClientCnxn: Socket
> connection established to XXXXXXXXXXXXXXXXXXXXX, initiating session
> 2011-02-03 10:37:44,767 DEBUG
> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU eviction
> started; Attempting to free 81.93 MB of total=696.25 MB
> 2011-02-03 10:37:44,784 DEBUG
> org.apache.hadoop.hbase.io.hfile.LruBlockCache: Block cache LRU eviction
> completed; freed=81.94 MB, total=614.81 MB, single=379.98 MB,
> multi=309.77 MB, memory=0 KB
> 2011-02-03 10:37:45,205 INFO org.apache.zookeeper.ClientCnxn: Unable to
> reconnect to ZooKeeper service, session 0x22db9f722501d93 has expired,
> closing socket connection
> 2011-02-03 10:37:45,206 INFO
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> entation:
> This client just lost it's session with ZooKeeper, trying to reconnect.
> 2011-02-03 10:37:45,453 INFO
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplem
> entation:
> Trying to reconnect to zookeeper
> 2011-02-03 10:37:45,206 INFO org.apache.zookeeper.ClientCnxn: Unable to
> reconnect to ZooKeeper service, session 0x12db9f722421ce3 has expired,
> closing socket connection
> gionserver:60020-0x22db9f722501d93 regionserver:60020-
> 0x22db9f722501d93
> received expired from ZooKeeper, aborting
> org.apache.zookeeper.KeeperException$SessionExpiredException:
> KeeperErrorCode = Session expired
>         at
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.connectionEvent(
> ZooKeeperWatcher.java:328)
>         at
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeep
> erWatcher.java:246)
>         at
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.ja
> va:530)
>         at
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:506)
> handled exception: org.apache.hadoop.hbase.YouAreDeadException: Server
> REPORT rejected; currently processing XXXXXXXXXXXX,60020,1296684296172
> as dead server
> org.apache.hadoop.hbase.YouAreDeadException:
> org.apache.hadoop.hbase.YouAreDeadException: Server REPORT rejected;
> currently processing XXXXXXXXXXXX,60020,1296684296172 as dead server
>         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> Method)
>         at
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
> AccessorImpl.java:39)
>         at
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
> structorAccessorImpl.java:27)
>         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>         at
> org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteExce
> ption.java:96)
>         at
> org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(Remote
> Exception.java:80)
>         at
> org.apache.hadoop.hbase.regionserver.HRegionServer.tryRegionServerRep
> ort(HRegionServer.java:729)
>         at
> org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.j
> ava:586)
>         at java.lang.Thread.run(Thread.java:619)
> 
> 
> 2011-02-03T09:53:35.165-0800: 71320.785: [GC 71320.785: [ParNew
> (promotion
> failed): 5568K->5568K(5568K), 0.4384530 secs]71321.224:
> [CMS2011-02-03T09:53:45.111-0800: 71330.731: [CMS-concurrent-mark:
> 17.511/51.564 secs] [Times: user=38.72 sys=5.67, real=51.60 secs]
> 
> 
> 
> Thanks,
> Charan