You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Gaojinchao <ga...@huawei.com> on 2011/05/09 05:09:23 UTC

Hmaster is OutOfMemory

Hbase version 0.90.2:
Hmaster has 8G memory,  It seems like not enough ? why it needs so much memory?(50K region)

Other issue. Log is error:
see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9 should be see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A8

Hmaster logs:

2011-05-06 19:31:09,924 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 2f19f33ae3f21ac4cb681f1662767d0c with OFFLINE state
2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 76166ms instead of 60000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16697ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Forcing OFFLINE; was=ufdr,211007,1304669377398.696f124cc6ff82302f735c8413c6ac0b. state=CLOSED, ts=1304681364406
2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 696f124cc6ff82302f735c8413c6ac0b with OFFLINE state
2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Opened region ufdr,071415,1304668656420.aa026fbb27a25b0fe54039c00108dad6. on 157-5-100-9,20020,1304678135900
2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for 7a75bac2028fba1529075225a3755c4c; deleting unassigned node
2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for 7a75bac2028fba1529075225a3755c4c that is in expected state RS_ZK_REGION_OPENED
2011-05-06 19:31:38,986 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 12948ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:31:39,059 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Sent CLOSE to serverName=158-1-91-101,20020,1304663941215, load=(requests=0, regions=5374, usedHeap=79, maxHeap=8165) for region ufdr,082288,1304668716520.144efd9c886f5567abc9a229eb380c76.
2011-05-06 19:31:39,059 INFO org.apache.hadoop.hbase.master.HMaster: balance hri=ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22., src=158-1-91-101,20020,1304663941215, dest=157-5-100-9,20020,1304678135900

2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-2011-05-06 19:31:54,741 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15681ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:32:11,419 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16574ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:32:23,176 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 11756ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:32:23,245 INFO org.apache.zookeeper.ClientCnxn: Unable to read additional data from server sessionid 0x12fc3a17c070054, likely server has closed socket, closing socket connection and attempting reconnect
2011-05-06 19:32:36,902 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 13657ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:35:44,084 WARN org.apache.hadoop.hbase.zookeeper.ZKUtil: hconnection-0x12fc3a17c070054 Unable to set watcher on znode /hbase/root-region-server
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
	at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
	at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
	at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
	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:541)
	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
2011-05-06 19:35:44,096 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 78032ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:35:44,095 ERROR org.apache.hadoop.hbase.executor.EventHandler: Caught throwable while processing event RS_ZK_REGION_OPENED
java.lang.OutOfMemoryError: Java heap space
	at java.util.HashMap.addEntry(HashMap.java:753)
	at java.util.HashMap.put(HashMap.java:385)
	at java.util.HashMap.putAll(HashMap.java:524)
	at org.apache.hadoop.hbase.master.AssignmentManager.updateTimers(AssignmentManager.java:630)
	at org.apache.hadoop.hbase.master.AssignmentManager.regionOnline(AssignmentManager.java:607)
	at org.apache.hadoop.hbase.master.handler.OpenedRegionHandler.process(OpenedRegionHandler.java:97)
	at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:151)
	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
	at java.lang.Thread.run(Thread.java:662)
2011-05-06 19:35:59,981 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for accac1bddaa4bf102bcfa6506b19839d; deleting unassigned node
2011-05-06 19:35:44,093 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Starting unassignment of region ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22. (offlining)
2011-05-06 19:35:59,981 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
2011-05-06 19:35:59,981 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15884ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:35:44,097 ERROR org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: hconnection-0x12fc3a17c070054 Received unexpected KeeperException, re-throwing exception
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
	at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
	at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
	at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
	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:541)
	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)5-100-11/157.5.100.11:2181, initiating session
2011-05-06 19:36:16,570 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16488ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-5-100-11/157.5.100.11:2181, initiating session
2011-05-06 19:36:16,571 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 16489ms for sessionid 0x12fc3a17c070054, closing socket connection and attempting reconnect
2011-05-06 19:36:00,082 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for accac1bddaa4bf102bcfa6506b19839d that is in expected state RS_ZK_REGION_OPENED
2011-05-06 19:36:16,570 FATAL org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Unexpected exception during initialization, aborting
org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
	at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
	at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
	at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
	at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
	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:541)
	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
2011-05-06 19:36:16,572 INFO org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Reconnected successfully. This disconnect could have been caused by a network partition or a long-running GC pause, either way it's recommended that you verify your environment.
2011-05-06 19:36:16,572 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down
2011-05-06 19:36:16,572 WARN org.apache.hadoop.hbase.master.CatalogJanitor: Failed scan of catalog table
org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hbase.UnknownScannerException: Name: 3135625757230134514
	at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1794)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1782)
	at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
	at java.lang.reflect.Method.invoke(Method.java:597)
	at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:570)
	at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039)

	at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:771)
	at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
	at $Proxy6.next(Unknown Source)
	at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:264)
	at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:237)
	at org.apache.hadoop.hbase.master.CatalogJanitor.scan(CatalogJanitor.java:116)
	at org.apache.hadoop.hbase.master.CatalogJanitor.chore(CatalogJanitor.java:85)
	at org.apache.hadoop.hbase.Chore.run(Chore.java:66)

Re: Hmaster is OutOfMemory

Posted by Gaojinchao <ga...@huawei.com>.
If the cluster has 100K regions , restart cluster, Master will need a lot of memory.


-----邮件原件-----
发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
发送时间: 2011年5月10日 13:58
收件人: user@hbase.apache.org
主题: Re: Hmaster is OutOfMemory

2011/5/9 Gaojinchao <ga...@huawei.com>:
> Hbase version : 0.90.3RC0
>
> It happened when creating table with Regions
> I find master started needs so much memory when the cluster has 100K regions

Do you need to have 100k regions in the cluster Gao?  Or, you are just
testing how we do w/ 100k regions?


> It seems likes zkclientcnxn.
>
> It seems master assigned region need improve.
>
>
> top -c | grep 5834
> 5834 root      20   0 8875m 7.9g  11m S    2 50.5  33:53.19 /opt/jdk1.6.0_22/bin/java -Xmx8192m -ea -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode
>

You probably don't need CMSIncrementalMode if your hardware has >= 4 CPUs.

Where do you see heap used in the below?  I just see stats on your
heap config. and a snapshot of what is currently in use.  Seems to be
5G of your 8G heap (~60%).   If you do a full GC, does this go down?

In 0.90.x, HBase Master keeps an 'image' of the cluster in HMaster
RAM.  I'd doubt this takes up 5G but I haven't measured it so perhaps
it could.  Is this a problem for you Gao?  You do have a 100k regions.

St.Ack

> Heap Configuration:
>   MinHeapFreeRatio = 40
>   MaxHeapFreeRatio = 70
>   MaxHeapSize      = 8589934592 (8192.0MB)
>   NewSize          = 21757952 (20.75MB)
>   MaxNewSize       = 174456832 (166.375MB)
>   OldSize          = 65404928 (62.375MB)
>   NewRatio         = 7
>   SurvivorRatio    = 8
>   PermSize         = 21757952 (20.75MB)
>   MaxPermSize      = 88080384 (84.0MB)
>
> Heap Usage:
> New Generation (Eden + 1 Survivor Space):
>   capacity = 100335616 (95.6875MB)
>   used     = 47094720 (44.91302490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   46.93719127612671% used
> Eden Space:
>   capacity = 89194496 (85.0625MB)
>   used     = 35953600 (34.28802490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   40.30921369856723% used
> From Space:
>   capacity = 11141120 (10.625MB)
>   used     = 11141120 (10.625MB)
>   free     = 0 (0.0MB)
>   100.0% used
> To Space:
>   capacity = 11141120 (10.625MB)
>   used     = 0 (0.0MB)
>   free     = 11141120 (10.625MB)
>   0.0% used
> concurrent mark-sweep generation:
>   capacity = 8415477760 (8025.625MB)
>   used     = 5107249280 (4870.6524658203125MB)
>   free     = 3308228480 (3154.9725341796875MB)
>   60.68876213155128% used
> Perm Generation:
>   capacity = 31199232 (29.75390625MB)
>   used     = 18681784 (17.81633758544922MB)
>   free     = 12517448 (11.937568664550781MB)
>   59.87898676480241% used
>
>
> -----邮件原件-----
> 发件人: jdcryans@gmail.com [mailto:jdcryans@gmail.com] 代表 Jean-Daniel Cryans
> 发送时间: 2011年5月10日 1:20
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
>
> It looks like the master entered a GC loop of death (since there are a
> lot of "We slept 76166ms" messages) and finally died. Was it splitting
> logs? Did you get a heap dump? Did you inspect it and can you tell
> what was using all that space?
>
> Thx,
>
> J-D
>
> 2011/5/8 Gaojinchao <ga...@huawei.com>:
>> Hbase version 0.90.2:
>> Hmaster has 8G memory,  It seems like not enough ? why it needs so much memory?(50K region)
>>
>> Other issue. Log is error:
>> see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9 should be see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A8
>>
>> Hmaster logs:
>>
>> 2011-05-06 19:31:09,924 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 2f19f33ae3f21ac4cb681f1662767d0c with OFFLINE state
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 76166ms instead of 60000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16697ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Forcing OFFLINE; was=ufdr,211007,1304669377398.696f124cc6ff82302f735c8413c6ac0b. state=CLOSED, ts=1304681364406
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 696f124cc6ff82302f735c8413c6ac0b with OFFLINE state
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Opened region ufdr,071415,1304668656420.aa026fbb27a25b0fe54039c00108dad6. on 157-5-100-9,20020,1304678135900
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for 7a75bac2028fba1529075225a3755c4c; deleting unassigned node
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for 7a75bac2028fba1529075225a3755c4c that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:31:38,986 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 12948ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:39,059 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Sent CLOSE to serverName=158-1-91-101,20020,1304663941215, load=(requests=0, regions=5374, usedHeap=79, maxHeap=8165) for region ufdr,082288,1304668716520.144efd9c886f5567abc9a229eb380c76.
>> 2011-05-06 19:31:39,059 INFO org.apache.hadoop.hbase.master.HMaster: balance hri=ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22., src=158-1-91-101,20020,1304663941215, dest=157-5-100-9,20020,1304678135900
>>
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-2011-05-06 19:31:54,741 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15681ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:11,419 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16574ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,176 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 11756ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,245 INFO org.apache.zookeeper.ClientCnxn: Unable to read additional data from server sessionid 0x12fc3a17c070054, likely server has closed socket, closing socket connection and attempting reconnect
>> 2011-05-06 19:32:36,902 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 13657ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,084 WARN org.apache.hadoop.hbase.zookeeper.ZKUtil: hconnection-0x12fc3a17c070054 Unable to set watcher on znode /hbase/root-region-server
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:35:44,096 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 78032ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,095 ERROR org.apache.hadoop.hbase.executor.EventHandler: Caught throwable while processing event RS_ZK_REGION_OPENED
>> java.lang.OutOfMemoryError: Java heap space
>>        at java.util.HashMap.addEntry(HashMap.java:753)
>>        at java.util.HashMap.put(HashMap.java:385)
>>        at java.util.HashMap.putAll(HashMap.java:524)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.updateTimers(AssignmentManager.java:630)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.regionOnline(AssignmentManager.java:607)
>>        at org.apache.hadoop.hbase.master.handler.OpenedRegionHandler.process(OpenedRegionHandler.java:97)
>>        at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:151)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>        at java.lang.Thread.run(Thread.java:662)
>> 2011-05-06 19:35:59,981 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for accac1bddaa4bf102bcfa6506b19839d; deleting unassigned node
>> 2011-05-06 19:35:44,093 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Starting unassignment of region ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22. (offlining)
>> 2011-05-06 19:35:59,981 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:35:59,981 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15884ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,097 ERROR org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: hconnection-0x12fc3a17c070054 Received unexpected KeeperException, re-throwing exception
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,570 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16488ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,571 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 16489ms for sessionid 0x12fc3a17c070054, closing socket connection and attempting reconnect
>> 2011-05-06 19:36:00,082 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for accac1bddaa4bf102bcfa6506b19839d that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:36:16,570 FATAL org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Unexpected exception during initialization, aborting
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:36:16,572 INFO org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Reconnected successfully. This disconnect could have been caused by a network partition or a long-running GC pause, either way it's recommended that you verify your environment.
>> 2011-05-06 19:36:16,572 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down
>> 2011-05-06 19:36:16,572 WARN org.apache.hadoop.hbase.master.CatalogJanitor: Failed scan of catalog table
>> org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hbase.UnknownScannerException: Name: 3135625757230134514
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1794)
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1782)
>>        at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
>>        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>        at java.lang.reflect.Method.invoke(Method.java:597)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:570)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039)
>>
>>        at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:771)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
>>        at $Proxy6.next(Unknown Source)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:264)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:237)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.scan(CatalogJanitor.java:116)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.chore(CatalogJanitor.java:85)
>>        at org.apache.hadoop.hbase.Chore.run(Chore.java:66)
>>
>

答复: 答复: Hmaster is OutOfMemory

Posted by "Jack Zhang(jian)" <ja...@huawei.com>.
Ok, I try to do it.

Jian Zhang(Jack)

-----邮件原件-----
发件人: Stack [mailto:saint.ack@gmail.com] 
发送时间: 2011年5月13日 9:48
收件人: user@hbase.apache.org
抄送: user@hbase.apache.org
主题: Re: 答复: Hmaster is OutOfMemory

Have a go at fixing it.  If you do it there is some hope it will make it into the code base soon.

My English isn't too good either but it's better than my Chinese 

Stack



On May 12, 2011, at 18:04, "Jack Zhang(jian)" <ja...@huawei.com> wrote:

> Maybe I didn't descript the issue clearly(I hate English ^_^).Yes, we need on branch. I don't have a patch yet, but if need I can try my best to fix this issue.
> 
> -----邮件原件-----
> 发件人: Stack [mailto:saint.ack@gmail.com] 
> 发送时间: 2011年5月13日 8:46
> 收件人: user@hbase.apache.org
> 抄送: user@hbase.apache.org; Chenjian
> 主题: Re: Hmaster is OutOfMemory
> 
> Pardon my being slow but I think I now understand what you are getting at.  I took a look at a heap dump on one of our production servers which is carrying 10k regions.  I see retention of an hserverload per online region.  The count of hserverload hregionload instances retained can be regions * regions.   On my cluster I see retention of about .5 gigs.   On a cluster of 100k regions it would be a good bit worse.   This should be fixed in trunk.  Do you need a fix on branch?   Maybe you have a patch that mulls out load when hsi is added to online regions.  Would need to make sure we did not break balancer if we did this
> 
> Thanks for digging in here
> 
> Stack
> 
> 
> 
> On May 12, 2011, at 5:44, "Jack Zhang(jian)" <ja...@huawei.com> wrote:
> 
>> In our test cluster, 1 hmaster, 3 regionserver, but there are 1,481 HServerInfo instance.
>> So in hbase 0.90.2 , I think there are memory leak in hmaster.
>> 1. Regionsever reports it's load to hmaster period, so the corresponding Hserverinfo will be changed in ServerManager.onlineServers(line 244, 337 in ServerManager) .
>> 2. When hbase cluster start up, AssignmentManager will receive RS_ZK_REGION_OPENED event, then it will construct OpenedRegionHandler(line 431 in AssignmentManager) by the corresponding Hserverinfo instance refreshed by the Regionserver latest.
>> 3. Then OpenedRegionHandler will store this Hserverinfo into assignmentManager.regionOnline(line 97 in OpenedRegionHandler).
>> 
>> After regionserver reported load, hmaster always store the new hserverinfo instance to assignmentManager.regionOnline if region opened meanwhile.
>> So the more the region opened, the more memory will leak.
>> 
>> -----邮件原件-----
>> 发件人: Gaojinchao [mailto:gaojinchao@huawei.com] 
>> 发送时间: 2011年5月12日 15:24
>> 收件人: user@hbase.apache.org
>> 主题: Re: Hmaster is OutOfMemory
>> 
>> Thanks, Stack.
>> The heap dump is so big. I try to dig it and share the result. 
>> Then ,you can give me some suggestion.
>> 
>> 
>> Do you give me more suggestion about my cluster?
>> 
>> My application:
>> write operation with api put<list> is about 75k Puts/s( one put about 400 Bytes)
>> read opreation is rarely, but the latency time demands lower than 5s
>> 
>> machine:
>> cpu:    8 core 2.GHz
>> memory: 24G, Hbase use 8G
>> Disk:   2T*8 = 16T
>> 
>> node number: 13 nodes
>> 
>> dfs configure:
>> 
>> dfs.block.size 256M 
>> dfs.datanode.handler.count 10 
>> dfs.namenode.handler.count 30 
>> dfs.datanode.max.xcievers 2047 
>> dfs.support.append True 
>> 
>> hbase configure:
>> 
>> hbase.regionserver.handler.count 50 
>> hbase.regionserver.global.memstore.upperLimit 0.4 
>> hbase.regionserver.global.memstore.lowerLimit 0.35 
>> hbase.hregion.memstore.flush.size 128M 
>> hbase.hregion.max.filesize 512M 
>> hbase.client.scanner.caching 1 
>> hfile.block.cache.size 0.2 
>> hbase.hregion.memstore.block.multiplier 3 
>> hbase.hstore.blockingStoreFiles 10 
>> hbase.hstore.compaction.min.size 64M 
>> 
>> compress: gz
>> 
>> I am afraid of some problem:
>> 1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
>> 2, If a region server crashed, It could bring on the other region server crashed?
>> 
>> Can you give some suggestion about the hbase parameter and my cluster ?
>> 
>> -----邮件原件-----
>> 发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
>> 发送时间: 2011年5月10日 23:21
>> 收件人: user@hbase.apache.org
>> 主题: Re: Hmaster is OutOfMemory
>> 
>> 2011/5/9 Gaojinchao <ga...@huawei.com>:
>>> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
>>> In the future, If store several PB, It will more regions.
>> 
>> You might want to up the size of your regions to 4G (FB run w/ big
>> regions IIUC).
>> 
>> Do you want to put up a heap dump for me to take a look at.  That'd be
>> easier than me figuring time to try and replicate your scenario.
>> 
>> Thanks Gao,
>> St.Ack

Re: 答复: Hmaster is OutOfMemory

Posted by Stack <sa...@gmail.com>.
Have a go at fixing it.  If you do it there is some hope it will make it into the code base soon.

My English isn't too good either but it's better than my Chinese 

Stack



On May 12, 2011, at 18:04, "Jack Zhang(jian)" <ja...@huawei.com> wrote:

> Maybe I didn't descript the issue clearly(I hate English ^_^).Yes, we need on branch. I don't have a patch yet, but if need I can try my best to fix this issue.
> 
> -----邮件原件-----
> 发件人: Stack [mailto:saint.ack@gmail.com] 
> 发送时间: 2011年5月13日 8:46
> 收件人: user@hbase.apache.org
> 抄送: user@hbase.apache.org; Chenjian
> 主题: Re: Hmaster is OutOfMemory
> 
> Pardon my being slow but I think I now understand what you are getting at.  I took a look at a heap dump on one of our production servers which is carrying 10k regions.  I see retention of an hserverload per online region.  The count of hserverload hregionload instances retained can be regions * regions.   On my cluster I see retention of about .5 gigs.   On a cluster of 100k regions it would be a good bit worse.   This should be fixed in trunk.  Do you need a fix on branch?   Maybe you have a patch that mulls out load when hsi is added to online regions.  Would need to make sure we did not break balancer if we did this
> 
> Thanks for digging in here
> 
> Stack
> 
> 
> 
> On May 12, 2011, at 5:44, "Jack Zhang(jian)" <ja...@huawei.com> wrote:
> 
>> In our test cluster, 1 hmaster, 3 regionserver, but there are 1,481 HServerInfo instance.
>> So in hbase 0.90.2 , I think there are memory leak in hmaster.
>> 1. Regionsever reports it's load to hmaster period, so the corresponding Hserverinfo will be changed in ServerManager.onlineServers(line 244, 337 in ServerManager) .
>> 2. When hbase cluster start up, AssignmentManager will receive RS_ZK_REGION_OPENED event, then it will construct OpenedRegionHandler(line 431 in AssignmentManager) by the corresponding Hserverinfo instance refreshed by the Regionserver latest.
>> 3. Then OpenedRegionHandler will store this Hserverinfo into assignmentManager.regionOnline(line 97 in OpenedRegionHandler).
>> 
>> After regionserver reported load, hmaster always store the new hserverinfo instance to assignmentManager.regionOnline if region opened meanwhile.
>> So the more the region opened, the more memory will leak.
>> 
>> -----邮件原件-----
>> 发件人: Gaojinchao [mailto:gaojinchao@huawei.com] 
>> 发送时间: 2011年5月12日 15:24
>> 收件人: user@hbase.apache.org
>> 主题: Re: Hmaster is OutOfMemory
>> 
>> Thanks, Stack.
>> The heap dump is so big. I try to dig it and share the result. 
>> Then ,you can give me some suggestion.
>> 
>> 
>> Do you give me more suggestion about my cluster?
>> 
>> My application:
>> write operation with api put<list> is about 75k Puts/s( one put about 400 Bytes)
>> read opreation is rarely, but the latency time demands lower than 5s
>> 
>> machine:
>> cpu:    8 core 2.GHz
>> memory: 24G, Hbase use 8G
>> Disk:   2T*8 = 16T
>> 
>> node number: 13 nodes
>> 
>> dfs configure:
>> 
>> dfs.block.size 256M 
>> dfs.datanode.handler.count 10 
>> dfs.namenode.handler.count 30 
>> dfs.datanode.max.xcievers 2047 
>> dfs.support.append True 
>> 
>> hbase configure:
>> 
>> hbase.regionserver.handler.count 50 
>> hbase.regionserver.global.memstore.upperLimit 0.4 
>> hbase.regionserver.global.memstore.lowerLimit 0.35 
>> hbase.hregion.memstore.flush.size 128M 
>> hbase.hregion.max.filesize 512M 
>> hbase.client.scanner.caching 1 
>> hfile.block.cache.size 0.2 
>> hbase.hregion.memstore.block.multiplier 3 
>> hbase.hstore.blockingStoreFiles 10 
>> hbase.hstore.compaction.min.size 64M 
>> 
>> compress: gz
>> 
>> I am afraid of some problem:
>> 1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
>> 2, If a region server crashed, It could bring on the other region server crashed?
>> 
>> Can you give some suggestion about the hbase parameter and my cluster ?
>> 
>> -----邮件原件-----
>> 发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
>> 发送时间: 2011年5月10日 23:21
>> 收件人: user@hbase.apache.org
>> 主题: Re: Hmaster is OutOfMemory
>> 
>> 2011/5/9 Gaojinchao <ga...@huawei.com>:
>>> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
>>> In the future, If store several PB, It will more regions.
>> 
>> You might want to up the size of your regions to 4G (FB run w/ big
>> regions IIUC).
>> 
>> Do you want to put up a heap dump for me to take a look at.  That'd be
>> easier than me figuring time to try and replicate your scenario.
>> 
>> Thanks Gao,
>> St.Ack

答复: Hmaster is OutOfMemory

Posted by "Jack Zhang(jian)" <ja...@huawei.com>.
Maybe I didn't descript the issue clearly(I hate English ^_^).Yes, we need on branch. I don't have a patch yet, but if need I can try my best to fix this issue.

-----邮件原件-----
发件人: Stack [mailto:saint.ack@gmail.com] 
发送时间: 2011年5月13日 8:46
收件人: user@hbase.apache.org
抄送: user@hbase.apache.org; Chenjian
主题: Re: Hmaster is OutOfMemory

Pardon my being slow but I think I now understand what you are getting at.  I took a look at a heap dump on one of our production servers which is carrying 10k regions.  I see retention of an hserverload per online region.  The count of hserverload hregionload instances retained can be regions * regions.   On my cluster I see retention of about .5 gigs.   On a cluster of 100k regions it would be a good bit worse.   This should be fixed in trunk.  Do you need a fix on branch?   Maybe you have a patch that mulls out load when hsi is added to online regions.  Would need to make sure we did not break balancer if we did this

Thanks for digging in here

Stack



On May 12, 2011, at 5:44, "Jack Zhang(jian)" <ja...@huawei.com> wrote:

> In our test cluster, 1 hmaster, 3 regionserver, but there are 1,481 HServerInfo instance.
> So in hbase 0.90.2 , I think there are memory leak in hmaster.
> 1. Regionsever reports it's load to hmaster period, so the corresponding Hserverinfo will be changed in ServerManager.onlineServers(line 244, 337 in ServerManager) .
> 2. When hbase cluster start up, AssignmentManager will receive RS_ZK_REGION_OPENED event, then it will construct OpenedRegionHandler(line 431 in AssignmentManager) by the corresponding Hserverinfo instance refreshed by the Regionserver latest.
> 3. Then OpenedRegionHandler will store this Hserverinfo into assignmentManager.regionOnline(line 97 in OpenedRegionHandler).
> 
> After regionserver reported load, hmaster always store the new hserverinfo instance to assignmentManager.regionOnline if region opened meanwhile.
> So the more the region opened, the more memory will leak.
> 
> -----邮件原件-----
> 发件人: Gaojinchao [mailto:gaojinchao@huawei.com] 
> 发送时间: 2011年5月12日 15:24
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
> 
> Thanks, Stack.
> The heap dump is so big. I try to dig it and share the result. 
> Then ,you can give me some suggestion.
> 
> 
> Do you give me more suggestion about my cluster?
> 
> My application:
> write operation with api put<list> is about 75k Puts/s( one put about 400 Bytes)
> read opreation is rarely, but the latency time demands lower than 5s
> 
> machine:
> cpu:    8 core 2.GHz
> memory: 24G, Hbase use 8G
> Disk:   2T*8 = 16T
> 
> node number: 13 nodes
> 
> dfs configure:
> 
> dfs.block.size 256M 
> dfs.datanode.handler.count 10 
> dfs.namenode.handler.count 30 
> dfs.datanode.max.xcievers 2047 
> dfs.support.append True 
> 
> hbase configure:
> 
> hbase.regionserver.handler.count 50 
> hbase.regionserver.global.memstore.upperLimit 0.4 
> hbase.regionserver.global.memstore.lowerLimit 0.35 
> hbase.hregion.memstore.flush.size 128M 
> hbase.hregion.max.filesize 512M 
> hbase.client.scanner.caching 1 
> hfile.block.cache.size 0.2 
> hbase.hregion.memstore.block.multiplier 3 
> hbase.hstore.blockingStoreFiles 10 
> hbase.hstore.compaction.min.size 64M 
> 
> compress: gz
> 
> I am afraid of some problem:
> 1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
> 2, If a region server crashed, It could bring on the other region server crashed?
> 
> Can you give some suggestion about the hbase parameter and my cluster ?
> 
> -----邮件原件-----
> 发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
> 发送时间: 2011年5月10日 23:21
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
> 
> 2011/5/9 Gaojinchao <ga...@huawei.com>:
>> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
>> In the future, If store several PB, It will more regions.
> 
> You might want to up the size of your regions to 4G (FB run w/ big
> regions IIUC).
> 
> Do you want to put up a heap dump for me to take a look at.  That'd be
> easier than me figuring time to try and replicate your scenario.
> 
> Thanks Gao,
> St.Ack

Re: Hmaster is OutOfMemory

Posted by Stack <sa...@gmail.com>.
Pardon my being slow but I think I now understand what you are getting at.  I took a look at a heap dump on one of our production servers which is carrying 10k regions.  I see retention of an hserverload per online region.  The count of hserverload hregionload instances retained can be regions * regions.   On my cluster I see retention of about .5 gigs.   On a cluster of 100k regions it would be a good bit worse.   This should be fixed in trunk.  Do you need a fix on branch?   Maybe you have a patch that mulls out load when hsi is added to online regions.  Would need to make sure we did not break balancer if we did this

Thanks for digging in here

Stack



On May 12, 2011, at 5:44, "Jack Zhang(jian)" <ja...@huawei.com> wrote:

> In our test cluster, 1 hmaster, 3 regionserver, but there are 1,481 HServerInfo instance.
> So in hbase 0.90.2 , I think there are memory leak in hmaster.
> 1. Regionsever reports it's load to hmaster period, so the corresponding Hserverinfo will be changed in ServerManager.onlineServers(line 244, 337 in ServerManager) .
> 2. When hbase cluster start up, AssignmentManager will receive RS_ZK_REGION_OPENED event, then it will construct OpenedRegionHandler(line 431 in AssignmentManager) by the corresponding Hserverinfo instance refreshed by the Regionserver latest.
> 3. Then OpenedRegionHandler will store this Hserverinfo into assignmentManager.regionOnline(line 97 in OpenedRegionHandler).
> 
> After regionserver reported load, hmaster always store the new hserverinfo instance to assignmentManager.regionOnline if region opened meanwhile.
> So the more the region opened, the more memory will leak.
> 
> -----邮件原件-----
> 发件人: Gaojinchao [mailto:gaojinchao@huawei.com] 
> 发送时间: 2011年5月12日 15:24
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
> 
> Thanks, Stack.
> The heap dump is so big. I try to dig it and share the result. 
> Then ,you can give me some suggestion.
> 
> 
> Do you give me more suggestion about my cluster?
> 
> My application:
> write operation with api put<list> is about 75k Puts/s( one put about 400 Bytes)
> read opreation is rarely, but the latency time demands lower than 5s
> 
> machine:
> cpu:    8 core 2.GHz
> memory: 24G, Hbase use 8G
> Disk:   2T*8 = 16T
> 
> node number: 13 nodes
> 
> dfs configure:
> 
> dfs.block.size 256M 
> dfs.datanode.handler.count 10 
> dfs.namenode.handler.count 30 
> dfs.datanode.max.xcievers 2047 
> dfs.support.append True 
> 
> hbase configure:
> 
> hbase.regionserver.handler.count 50 
> hbase.regionserver.global.memstore.upperLimit 0.4 
> hbase.regionserver.global.memstore.lowerLimit 0.35 
> hbase.hregion.memstore.flush.size 128M 
> hbase.hregion.max.filesize 512M 
> hbase.client.scanner.caching 1 
> hfile.block.cache.size 0.2 
> hbase.hregion.memstore.block.multiplier 3 
> hbase.hstore.blockingStoreFiles 10 
> hbase.hstore.compaction.min.size 64M 
> 
> compress: gz
> 
> I am afraid of some problem:
> 1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
> 2, If a region server crashed, It could bring on the other region server crashed?
> 
> Can you give some suggestion about the hbase parameter and my cluster ?
> 
> -----邮件原件-----
> 发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
> 发送时间: 2011年5月10日 23:21
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
> 
> 2011/5/9 Gaojinchao <ga...@huawei.com>:
>> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
>> In the future, If store several PB, It will more regions.
> 
> You might want to up the size of your regions to 4G (FB run w/ big
> regions IIUC).
> 
> Do you want to put up a heap dump for me to take a look at.  That'd be
> easier than me figuring time to try and replicate your scenario.
> 
> Thanks Gao,
> St.Ack

Re: Hmaster is OutOfMemory

Posted by Stack <st...@duboce.net>.
As far as I can tell, what you describe below is the expected
behavior.  What would you suggest we do otherwise?  Do you find that
these HServerInfo instances are consuming lots of memory?

FYI, HServerInfo is purged from TRUNK.

Thanks Jean,
St.Ack

2011/5/12 Jack Zhang(jian) <ja...@huawei.com>:
> In our test cluster, 1 hmaster, 3 regionserver, but there are 1,481 HServerInfo instance.
> So in hbase 0.90.2 , I think there are memory leak in hmaster.
> 1. Regionsever reports it's load to hmaster period, so the corresponding Hserverinfo will be changed in ServerManager.onlineServers(line 244, 337 in ServerManager) .
> 2. When hbase cluster start up, AssignmentManager will receive RS_ZK_REGION_OPENED event, then it will construct OpenedRegionHandler(line 431 in AssignmentManager) by the corresponding Hserverinfo instance refreshed by the Regionserver latest.
> 3. Then OpenedRegionHandler will store this Hserverinfo into assignmentManager.regionOnline(line 97 in OpenedRegionHandler).
>
> After regionserver reported load, hmaster always store the new hserverinfo instance to assignmentManager.regionOnline if region opened meanwhile.
> So the more the region opened, the more memory will leak.
>
> -----邮件原件-----
> 发件人: Gaojinchao [mailto:gaojinchao@huawei.com]
> 发送时间: 2011年5月12日 15:24
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
>
> Thanks, Stack.
> The heap dump is so big. I try to dig it and share the result.
> Then ,you can give me some suggestion.
>
>
> Do you give me more suggestion about my cluster?
>
> My application:
> write operation with api put<list> is about 75k Puts/s( one put about 400 Bytes)
> read opreation is rarely, but the latency time demands lower than 5s
>
> machine:
> cpu:    8 core 2.GHz
> memory: 24G, Hbase use 8G
> Disk:   2T*8 = 16T
>
> node number: 13 nodes
>
> dfs configure:
>
> dfs.block.size 256M
> dfs.datanode.handler.count 10
> dfs.namenode.handler.count 30
> dfs.datanode.max.xcievers 2047
> dfs.support.append True
>
> hbase configure:
>
> hbase.regionserver.handler.count 50
> hbase.regionserver.global.memstore.upperLimit 0.4
> hbase.regionserver.global.memstore.lowerLimit 0.35
> hbase.hregion.memstore.flush.size 128M
> hbase.hregion.max.filesize 512M
> hbase.client.scanner.caching 1
> hfile.block.cache.size 0.2
> hbase.hregion.memstore.block.multiplier 3
> hbase.hstore.blockingStoreFiles 10
> hbase.hstore.compaction.min.size 64M
>
> compress: gz
>
> I am afraid of some problem:
> 1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
> 2, If a region server crashed, It could bring on the other region server crashed?
>
> Can you give some suggestion about the hbase parameter and my cluster ?
>
> -----邮件原件-----
> 发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
> 发送时间: 2011年5月10日 23:21
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
>
> 2011/5/9 Gaojinchao <ga...@huawei.com>:
>> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
>> In the future, If store several PB, It will more regions.
>
> You might want to up the size of your regions to 4G (FB run w/ big
> regions IIUC).
>
> Do you want to put up a heap dump for me to take a look at.  That'd be
> easier than me figuring time to try and replicate your scenario.
>
> Thanks Gao,
> St.Ack
>

re: Hmaster is OutOfMemory

Posted by "Jack Zhang(jian)" <ja...@huawei.com>.
In our test cluster, 1 hmaster, 3 regionserver, but there are 1,481 HServerInfo instance.
So in hbase 0.90.2 , I think there are memory leak in hmaster.
1. Regionsever reports it's load to hmaster period, so the corresponding Hserverinfo will be changed in ServerManager.onlineServers(line 244, 337 in ServerManager) .
2. When hbase cluster start up, AssignmentManager will receive RS_ZK_REGION_OPENED event, then it will construct OpenedRegionHandler(line 431 in AssignmentManager) by the corresponding Hserverinfo instance refreshed by the Regionserver latest.
3. Then OpenedRegionHandler will store this Hserverinfo into assignmentManager.regionOnline(line 97 in OpenedRegionHandler).

After regionserver reported load, hmaster always store the new hserverinfo instance to assignmentManager.regionOnline if region opened meanwhile.
So the more the region opened, the more memory will leak.

-----邮件原件-----
发件人: Gaojinchao [mailto:gaojinchao@huawei.com] 
发送时间: 2011年5月12日 15:24
收件人: user@hbase.apache.org
主题: Re: Hmaster is OutOfMemory

Thanks, Stack.
The heap dump is so big. I try to dig it and share the result. 
Then ,you can give me some suggestion.


Do you give me more suggestion about my cluster?

My application:
write operation with api put<list> is about 75k Puts/s( one put about 400 Bytes)
read opreation is rarely, but the latency time demands lower than 5s

machine:
cpu:    8 core 2.GHz
memory: 24G, Hbase use 8G
Disk:   2T*8 = 16T

node number: 13 nodes

dfs configure:

dfs.block.size 256M 
dfs.datanode.handler.count 10 
dfs.namenode.handler.count 30 
dfs.datanode.max.xcievers 2047 
dfs.support.append True 
 
hbase configure:

hbase.regionserver.handler.count 50 
hbase.regionserver.global.memstore.upperLimit 0.4 
hbase.regionserver.global.memstore.lowerLimit 0.35 
hbase.hregion.memstore.flush.size 128M 
hbase.hregion.max.filesize 512M 
hbase.client.scanner.caching 1 
hfile.block.cache.size 0.2 
hbase.hregion.memstore.block.multiplier 3 
hbase.hstore.blockingStoreFiles 10 
hbase.hstore.compaction.min.size 64M 

compress: gz

I am afraid of some problem:
1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
2, If a region server crashed, It could bring on the other region server crashed?

Can you give some suggestion about the hbase parameter and my cluster ?

-----邮件原件-----
发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
发送时间: 2011年5月10日 23:21
收件人: user@hbase.apache.org
主题: Re: Hmaster is OutOfMemory

2011/5/9 Gaojinchao <ga...@huawei.com>:
> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
> In the future, If store several PB, It will more regions.

You might want to up the size of your regions to 4G (FB run w/ big
regions IIUC).

Do you want to put up a heap dump for me to take a look at.  That'd be
easier than me figuring time to try and replicate your scenario.

Thanks Gao,
St.Ack

Re: Hmaster is OutOfMemory

Posted by Gaojinchao <ga...@huawei.com>.
Hi , Thanks for your share.
In my machine, The disk has 14T~16T. We need use all disks.
So I want to solve it by Key design.
eg: 
A region server has 14T capability ( more than 14K regions )
So, I can make 1000 or less active regions for certain time.
First day: 0~1000
Second day: 1001~2000
..............

It seems some issues for region server restart.
The regions will be resigned. So, the balance is very importance

Other issue is cluster restart, How can these regions be resigned quickly.
I change the parameter hbase.regionserver.executor.openregion.threads and hbase.master.executor.openregion.threads.
I am testing it.

Do you think my scheme solution is feasibility? Thanks.


-----邮件原件-----
发件人: im_gumby@hotmail.com [mailto:im_gumby@hotmail.com] 代表 Michel Segel
发送时间: 2011年5月12日 18:47
收件人: user@hbase.apache.org
主题: Re: Hmaster is OutOfMemory

> 
> On May 12, 2011, at 2:24 AM, Gaojinchao <ga...@huawei.com> wrote:
> 
> I am afraid of some problem:
> 1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
> 2, If a region server crashed, It could bring on the other region server crashed?
> 
> Can you give some suggestion about the hbase parameter and my cluster ?


Hi,

From what everyone I've talked with, a rule of thumb is to have no more than 1000 regions per region server.
You can set this on a per table basis...

If you do the math, a 2GB max file size would mean that each node would then be able to have 2TB of hbase storage within 1000 regions per region server.

I'm testing this... But it takes a while to generate the data for the test...
The goal is to see how 4GB files work and to see the impact on the cluster.
At 4GB max files... 1000 regions would be 4 TB per node ... 

Note YMMV and I'm still testing. I'm sure someone with more experience has some better advice...


Sent from a remote device. Please excuse any typos...

Mike Segel


Re: Hmaster is OutOfMemory

Posted by Michel Segel <mi...@hotmail.com>.
> 
> On May 12, 2011, at 2:24 AM, Gaojinchao <ga...@huawei.com> wrote:
> 
> I am afraid of some problem:
> 1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
> 2, If a region server crashed, It could bring on the other region server crashed?
> 
> Can you give some suggestion about the hbase parameter and my cluster ?


Hi,

From what everyone I've talked with, a rule of thumb is to have no more than 1000 regions per region server.
You can set this on a per table basis...

If you do the math, a 2GB max file size would mean that each node would then be able to have 2TB of hbase storage within 1000 regions per region server.

I'm testing this... But it takes a while to generate the data for the test...
The goal is to see how 4GB files work and to see the impact on the cluster.
At 4GB max files... 1000 regions would be 4 TB per node ... 

Note YMMV and I'm still testing. I'm sure someone with more experience has some better advice...


Sent from a remote device. Please excuse any typos...

Mike Segel


Re: Hmaster is OutOfMemory

Posted by Gaojinchao <ga...@huawei.com>.
Thanks, Stack.
The heap dump is so big. I try to dig it and share the result. 
Then ,you can give me some suggestion.


Do you give me more suggestion about my cluster?

My application:
write operation with api put<list> is about 75k Puts/s( one put about 400 Bytes)
read opreation is rarely, but the latency time demands lower than 5s

machine:
cpu:    8 core 2.GHz
memory: 24G, Hbase use 8G
Disk:   2T*8 = 16T

node number: 13 nodes

dfs configure:

dfs.block.size 256M 
dfs.datanode.handler.count 10 
dfs.namenode.handler.count 30 
dfs.datanode.max.xcievers 2047 
dfs.support.append True 
 
hbase configure:

hbase.regionserver.handler.count 50 
hbase.regionserver.global.memstore.upperLimit 0.4 
hbase.regionserver.global.memstore.lowerLimit 0.35 
hbase.hregion.memstore.flush.size 128M 
hbase.hregion.max.filesize 512M 
hbase.client.scanner.caching 1 
hfile.block.cache.size 0.2 
hbase.hregion.memstore.block.multiplier 3 
hbase.hstore.blockingStoreFiles 10 
hbase.hstore.compaction.min.size 64M 

compress: gz

I am afraid of some problem:
1, One region server has about 12k reigons or more, if up the hregion.max.filesize , parallel scalability will be low and affect scan latency .
2, If a region server crashed, It could bring on the other region server crashed?

Can you give some suggestion about the hbase parameter and my cluster ?

-----邮件原件-----
发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
发送时间: 2011年5月10日 23:21
收件人: user@hbase.apache.org
主题: Re: Hmaster is OutOfMemory

2011/5/9 Gaojinchao <ga...@huawei.com>:
> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
> In the future, If store several PB, It will more regions.

You might want to up the size of your regions to 4G (FB run w/ big
regions IIUC).

Do you want to put up a heap dump for me to take a look at.  That'd be
easier than me figuring time to try and replicate your scenario.

Thanks Gao,
St.Ack

Re: Hmaster is OutOfMemory

Posted by Stack <st...@duboce.net>.
2011/5/9 Gaojinchao <ga...@huawei.com>:
> My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
> In the future, If store several PB, It will more regions.

You might want to up the size of your regions to 4G (FB run w/ big
regions IIUC).

Do you want to put up a heap dump for me to take a look at.  That'd be
easier than me figuring time to try and replicate your scenario.

Thanks Gao,
St.Ack

Re: Hmaster is OutOfMemory

Posted by Gaojinchao <ga...@huawei.com>.
My first cluster needs save 147 TB data. If one region has 512M or 1 GB, It will be 300 K regions or 147K regions.
In the future, If store several PB, It will more regions.
I have a silly question.
how do it in Facebook? 

-----邮件原件-----
发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
发送时间: 2011年5月10日 13:58
收件人: user@hbase.apache.org
主题: Re: Hmaster is OutOfMemory

2011/5/9 Gaojinchao <ga...@huawei.com>:
> Hbase version : 0.90.3RC0
>
> It happened when creating table with Regions
> I find master started needs so much memory when the cluster has 100K regions

Do you need to have 100k regions in the cluster Gao?  Or, you are just
testing how we do w/ 100k regions?


> It seems likes zkclientcnxn.
>
> It seems master assigned region need improve.
>
>
> top -c | grep 5834
> 5834 root      20   0 8875m 7.9g  11m S    2 50.5  33:53.19 /opt/jdk1.6.0_22/bin/java -Xmx8192m -ea -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode
>

You probably don't need CMSIncrementalMode if your hardware has >= 4 CPUs.

Where do you see heap used in the below?  I just see stats on your
heap config. and a snapshot of what is currently in use.  Seems to be
5G of your 8G heap (~60%).   If you do a full GC, does this go down?

In 0.90.x, HBase Master keeps an 'image' of the cluster in HMaster
RAM.  I'd doubt this takes up 5G but I haven't measured it so perhaps
it could.  Is this a problem for you Gao?  You do have a 100k regions.

St.Ack

> Heap Configuration:
>   MinHeapFreeRatio = 40
>   MaxHeapFreeRatio = 70
>   MaxHeapSize      = 8589934592 (8192.0MB)
>   NewSize          = 21757952 (20.75MB)
>   MaxNewSize       = 174456832 (166.375MB)
>   OldSize          = 65404928 (62.375MB)
>   NewRatio         = 7
>   SurvivorRatio    = 8
>   PermSize         = 21757952 (20.75MB)
>   MaxPermSize      = 88080384 (84.0MB)
>
> Heap Usage:
> New Generation (Eden + 1 Survivor Space):
>   capacity = 100335616 (95.6875MB)
>   used     = 47094720 (44.91302490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   46.93719127612671% used
> Eden Space:
>   capacity = 89194496 (85.0625MB)
>   used     = 35953600 (34.28802490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   40.30921369856723% used
> From Space:
>   capacity = 11141120 (10.625MB)
>   used     = 11141120 (10.625MB)
>   free     = 0 (0.0MB)
>   100.0% used
> To Space:
>   capacity = 11141120 (10.625MB)
>   used     = 0 (0.0MB)
>   free     = 11141120 (10.625MB)
>   0.0% used
> concurrent mark-sweep generation:
>   capacity = 8415477760 (8025.625MB)
>   used     = 5107249280 (4870.6524658203125MB)
>   free     = 3308228480 (3154.9725341796875MB)
>   60.68876213155128% used
> Perm Generation:
>   capacity = 31199232 (29.75390625MB)
>   used     = 18681784 (17.81633758544922MB)
>   free     = 12517448 (11.937568664550781MB)
>   59.87898676480241% used
>
>
> -----邮件原件-----
> 发件人: jdcryans@gmail.com [mailto:jdcryans@gmail.com] 代表 Jean-Daniel Cryans
> 发送时间: 2011年5月10日 1:20
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
>
> It looks like the master entered a GC loop of death (since there are a
> lot of "We slept 76166ms" messages) and finally died. Was it splitting
> logs? Did you get a heap dump? Did you inspect it and can you tell
> what was using all that space?
>
> Thx,
>
> J-D
>
> 2011/5/8 Gaojinchao <ga...@huawei.com>:
>> Hbase version 0.90.2:
>> Hmaster has 8G memory,  It seems like not enough ? why it needs so much memory?(50K region)
>>
>> Other issue. Log is error:
>> see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9 should be see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A8
>>
>> Hmaster logs:
>>
>> 2011-05-06 19:31:09,924 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 2f19f33ae3f21ac4cb681f1662767d0c with OFFLINE state
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 76166ms instead of 60000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16697ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Forcing OFFLINE; was=ufdr,211007,1304669377398.696f124cc6ff82302f735c8413c6ac0b. state=CLOSED, ts=1304681364406
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 696f124cc6ff82302f735c8413c6ac0b with OFFLINE state
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Opened region ufdr,071415,1304668656420.aa026fbb27a25b0fe54039c00108dad6. on 157-5-100-9,20020,1304678135900
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for 7a75bac2028fba1529075225a3755c4c; deleting unassigned node
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for 7a75bac2028fba1529075225a3755c4c that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:31:38,986 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 12948ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:39,059 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Sent CLOSE to serverName=158-1-91-101,20020,1304663941215, load=(requests=0, regions=5374, usedHeap=79, maxHeap=8165) for region ufdr,082288,1304668716520.144efd9c886f5567abc9a229eb380c76.
>> 2011-05-06 19:31:39,059 INFO org.apache.hadoop.hbase.master.HMaster: balance hri=ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22., src=158-1-91-101,20020,1304663941215, dest=157-5-100-9,20020,1304678135900
>>
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-2011-05-06 19:31:54,741 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15681ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:11,419 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16574ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,176 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 11756ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,245 INFO org.apache.zookeeper.ClientCnxn: Unable to read additional data from server sessionid 0x12fc3a17c070054, likely server has closed socket, closing socket connection and attempting reconnect
>> 2011-05-06 19:32:36,902 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 13657ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,084 WARN org.apache.hadoop.hbase.zookeeper.ZKUtil: hconnection-0x12fc3a17c070054 Unable to set watcher on znode /hbase/root-region-server
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:35:44,096 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 78032ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,095 ERROR org.apache.hadoop.hbase.executor.EventHandler: Caught throwable while processing event RS_ZK_REGION_OPENED
>> java.lang.OutOfMemoryError: Java heap space
>>        at java.util.HashMap.addEntry(HashMap.java:753)
>>        at java.util.HashMap.put(HashMap.java:385)
>>        at java.util.HashMap.putAll(HashMap.java:524)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.updateTimers(AssignmentManager.java:630)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.regionOnline(AssignmentManager.java:607)
>>        at org.apache.hadoop.hbase.master.handler.OpenedRegionHandler.process(OpenedRegionHandler.java:97)
>>        at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:151)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>        at java.lang.Thread.run(Thread.java:662)
>> 2011-05-06 19:35:59,981 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for accac1bddaa4bf102bcfa6506b19839d; deleting unassigned node
>> 2011-05-06 19:35:44,093 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Starting unassignment of region ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22. (offlining)
>> 2011-05-06 19:35:59,981 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:35:59,981 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15884ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,097 ERROR org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: hconnection-0x12fc3a17c070054 Received unexpected KeeperException, re-throwing exception
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,570 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16488ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,571 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 16489ms for sessionid 0x12fc3a17c070054, closing socket connection and attempting reconnect
>> 2011-05-06 19:36:00,082 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for accac1bddaa4bf102bcfa6506b19839d that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:36:16,570 FATAL org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Unexpected exception during initialization, aborting
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:36:16,572 INFO org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Reconnected successfully. This disconnect could have been caused by a network partition or a long-running GC pause, either way it's recommended that you verify your environment.
>> 2011-05-06 19:36:16,572 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down
>> 2011-05-06 19:36:16,572 WARN org.apache.hadoop.hbase.master.CatalogJanitor: Failed scan of catalog table
>> org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hbase.UnknownScannerException: Name: 3135625757230134514
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1794)
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1782)
>>        at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
>>        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>        at java.lang.reflect.Method.invoke(Method.java:597)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:570)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039)
>>
>>        at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:771)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
>>        at $Proxy6.next(Unknown Source)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:264)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:237)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.scan(CatalogJanitor.java:116)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.chore(CatalogJanitor.java:85)
>>        at org.apache.hadoop.hbase.Chore.run(Chore.java:66)
>>
>

Re: Hmaster is OutOfMemory

Posted by Stack <st...@duboce.net>.
2011/5/9 Gaojinchao <ga...@huawei.com>:
> Hbase version : 0.90.3RC0
>
> It happened when creating table with Regions
> I find master started needs so much memory when the cluster has 100K regions

Do you need to have 100k regions in the cluster Gao?  Or, you are just
testing how we do w/ 100k regions?


> It seems likes zkclientcnxn.
>
> It seems master assigned region need improve.
>
>
> top -c | grep 5834
> 5834 root      20   0 8875m 7.9g  11m S    2 50.5  33:53.19 /opt/jdk1.6.0_22/bin/java -Xmx8192m -ea -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode
>

You probably don't need CMSIncrementalMode if your hardware has >= 4 CPUs.

Where do you see heap used in the below?  I just see stats on your
heap config. and a snapshot of what is currently in use.  Seems to be
5G of your 8G heap (~60%).   If you do a full GC, does this go down?

In 0.90.x, HBase Master keeps an 'image' of the cluster in HMaster
RAM.  I'd doubt this takes up 5G but I haven't measured it so perhaps
it could.  Is this a problem for you Gao?  You do have a 100k regions.

St.Ack

> Heap Configuration:
>   MinHeapFreeRatio = 40
>   MaxHeapFreeRatio = 70
>   MaxHeapSize      = 8589934592 (8192.0MB)
>   NewSize          = 21757952 (20.75MB)
>   MaxNewSize       = 174456832 (166.375MB)
>   OldSize          = 65404928 (62.375MB)
>   NewRatio         = 7
>   SurvivorRatio    = 8
>   PermSize         = 21757952 (20.75MB)
>   MaxPermSize      = 88080384 (84.0MB)
>
> Heap Usage:
> New Generation (Eden + 1 Survivor Space):
>   capacity = 100335616 (95.6875MB)
>   used     = 47094720 (44.91302490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   46.93719127612671% used
> Eden Space:
>   capacity = 89194496 (85.0625MB)
>   used     = 35953600 (34.28802490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   40.30921369856723% used
> From Space:
>   capacity = 11141120 (10.625MB)
>   used     = 11141120 (10.625MB)
>   free     = 0 (0.0MB)
>   100.0% used
> To Space:
>   capacity = 11141120 (10.625MB)
>   used     = 0 (0.0MB)
>   free     = 11141120 (10.625MB)
>   0.0% used
> concurrent mark-sweep generation:
>   capacity = 8415477760 (8025.625MB)
>   used     = 5107249280 (4870.6524658203125MB)
>   free     = 3308228480 (3154.9725341796875MB)
>   60.68876213155128% used
> Perm Generation:
>   capacity = 31199232 (29.75390625MB)
>   used     = 18681784 (17.81633758544922MB)
>   free     = 12517448 (11.937568664550781MB)
>   59.87898676480241% used
>
>
> -----邮件原件-----
> 发件人: jdcryans@gmail.com [mailto:jdcryans@gmail.com] 代表 Jean-Daniel Cryans
> 发送时间: 2011年5月10日 1:20
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
>
> It looks like the master entered a GC loop of death (since there are a
> lot of "We slept 76166ms" messages) and finally died. Was it splitting
> logs? Did you get a heap dump? Did you inspect it and can you tell
> what was using all that space?
>
> Thx,
>
> J-D
>
> 2011/5/8 Gaojinchao <ga...@huawei.com>:
>> Hbase version 0.90.2:
>> Hmaster has 8G memory,  It seems like not enough ? why it needs so much memory?(50K region)
>>
>> Other issue. Log is error:
>> see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9 should be see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A8
>>
>> Hmaster logs:
>>
>> 2011-05-06 19:31:09,924 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 2f19f33ae3f21ac4cb681f1662767d0c with OFFLINE state
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 76166ms instead of 60000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16697ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Forcing OFFLINE; was=ufdr,211007,1304669377398.696f124cc6ff82302f735c8413c6ac0b. state=CLOSED, ts=1304681364406
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 696f124cc6ff82302f735c8413c6ac0b with OFFLINE state
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Opened region ufdr,071415,1304668656420.aa026fbb27a25b0fe54039c00108dad6. on 157-5-100-9,20020,1304678135900
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for 7a75bac2028fba1529075225a3755c4c; deleting unassigned node
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for 7a75bac2028fba1529075225a3755c4c that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:31:38,986 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 12948ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:39,059 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Sent CLOSE to serverName=158-1-91-101,20020,1304663941215, load=(requests=0, regions=5374, usedHeap=79, maxHeap=8165) for region ufdr,082288,1304668716520.144efd9c886f5567abc9a229eb380c76.
>> 2011-05-06 19:31:39,059 INFO org.apache.hadoop.hbase.master.HMaster: balance hri=ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22., src=158-1-91-101,20020,1304663941215, dest=157-5-100-9,20020,1304678135900
>>
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-2011-05-06 19:31:54,741 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15681ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:11,419 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16574ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,176 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 11756ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,245 INFO org.apache.zookeeper.ClientCnxn: Unable to read additional data from server sessionid 0x12fc3a17c070054, likely server has closed socket, closing socket connection and attempting reconnect
>> 2011-05-06 19:32:36,902 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 13657ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,084 WARN org.apache.hadoop.hbase.zookeeper.ZKUtil: hconnection-0x12fc3a17c070054 Unable to set watcher on znode /hbase/root-region-server
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:35:44,096 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 78032ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,095 ERROR org.apache.hadoop.hbase.executor.EventHandler: Caught throwable while processing event RS_ZK_REGION_OPENED
>> java.lang.OutOfMemoryError: Java heap space
>>        at java.util.HashMap.addEntry(HashMap.java:753)
>>        at java.util.HashMap.put(HashMap.java:385)
>>        at java.util.HashMap.putAll(HashMap.java:524)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.updateTimers(AssignmentManager.java:630)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.regionOnline(AssignmentManager.java:607)
>>        at org.apache.hadoop.hbase.master.handler.OpenedRegionHandler.process(OpenedRegionHandler.java:97)
>>        at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:151)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>        at java.lang.Thread.run(Thread.java:662)
>> 2011-05-06 19:35:59,981 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for accac1bddaa4bf102bcfa6506b19839d; deleting unassigned node
>> 2011-05-06 19:35:44,093 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Starting unassignment of region ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22. (offlining)
>> 2011-05-06 19:35:59,981 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:35:59,981 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15884ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,097 ERROR org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: hconnection-0x12fc3a17c070054 Received unexpected KeeperException, re-throwing exception
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,570 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16488ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,571 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 16489ms for sessionid 0x12fc3a17c070054, closing socket connection and attempting reconnect
>> 2011-05-06 19:36:00,082 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for accac1bddaa4bf102bcfa6506b19839d that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:36:16,570 FATAL org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Unexpected exception during initialization, aborting
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:36:16,572 INFO org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Reconnected successfully. This disconnect could have been caused by a network partition or a long-running GC pause, either way it's recommended that you verify your environment.
>> 2011-05-06 19:36:16,572 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down
>> 2011-05-06 19:36:16,572 WARN org.apache.hadoop.hbase.master.CatalogJanitor: Failed scan of catalog table
>> org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hbase.UnknownScannerException: Name: 3135625757230134514
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1794)
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1782)
>>        at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
>>        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>        at java.lang.reflect.Method.invoke(Method.java:597)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:570)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039)
>>
>>        at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:771)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
>>        at $Proxy6.next(Unknown Source)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:264)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:237)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.scan(CatalogJanitor.java:116)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.chore(CatalogJanitor.java:85)
>>        at org.apache.hadoop.hbase.Chore.run(Chore.java:66)
>>
>

Re: Hmaster is OutOfMemory

Posted by Stack <st...@duboce.net>.
Want to dump the heap and put it somewhere so we can pull it and take
a look Gao?
St.Ack

2011/5/9 Gaojinchao <ga...@huawei.com>:
> Hbase version : 0.90.3RC0
>
> It happened when creating table with Regions
> I find master started needs so much memory when the cluster has 100K regions
> It seems likes zkclientcnxn.
>
> It seems master assigned region need improve.
>
>
> top -c | grep 5834
> 5834 root      20   0 8875m 7.9g  11m S    2 50.5  33:53.19 /opt/jdk1.6.0_22/bin/java -Xmx8192m -ea -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode
>
> Attaching to process ID 5834, please wait...
> Debugger attached successfully.
> Server compiler detected.
> JVM version is 17.1-b03
>
> using parallel threads in the new generation.
> using thread-local object allocation.
> Concurrent Mark-Sweep GC
>
> Heap Configuration:
>   MinHeapFreeRatio = 40
>   MaxHeapFreeRatio = 70
>   MaxHeapSize      = 8589934592 (8192.0MB)
>   NewSize          = 21757952 (20.75MB)
>   MaxNewSize       = 174456832 (166.375MB)
>   OldSize          = 65404928 (62.375MB)
>   NewRatio         = 7
>   SurvivorRatio    = 8
>   PermSize         = 21757952 (20.75MB)
>   MaxPermSize      = 88080384 (84.0MB)
>
> Heap Usage:
> New Generation (Eden + 1 Survivor Space):
>   capacity = 100335616 (95.6875MB)
>   used     = 47094720 (44.91302490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   46.93719127612671% used
> Eden Space:
>   capacity = 89194496 (85.0625MB)
>   used     = 35953600 (34.28802490234375MB)
>   free     = 53240896 (50.77447509765625MB)
>   40.30921369856723% used
> From Space:
>   capacity = 11141120 (10.625MB)
>   used     = 11141120 (10.625MB)
>   free     = 0 (0.0MB)
>   100.0% used
> To Space:
>   capacity = 11141120 (10.625MB)
>   used     = 0 (0.0MB)
>   free     = 11141120 (10.625MB)
>   0.0% used
> concurrent mark-sweep generation:
>   capacity = 8415477760 (8025.625MB)
>   used     = 5107249280 (4870.6524658203125MB)
>   free     = 3308228480 (3154.9725341796875MB)
>   60.68876213155128% used
> Perm Generation:
>   capacity = 31199232 (29.75390625MB)
>   used     = 18681784 (17.81633758544922MB)
>   free     = 12517448 (11.937568664550781MB)
>   59.87898676480241% used
>
>
> -----邮件原件-----
> 发件人: jdcryans@gmail.com [mailto:jdcryans@gmail.com] 代表 Jean-Daniel Cryans
> 发送时间: 2011年5月10日 1:20
> 收件人: user@hbase.apache.org
> 主题: Re: Hmaster is OutOfMemory
>
> It looks like the master entered a GC loop of death (since there are a
> lot of "We slept 76166ms" messages) and finally died. Was it splitting
> logs? Did you get a heap dump? Did you inspect it and can you tell
> what was using all that space?
>
> Thx,
>
> J-D
>
> 2011/5/8 Gaojinchao <ga...@huawei.com>:
>> Hbase version 0.90.2:
>> Hmaster has 8G memory,  It seems like not enough ? why it needs so much memory?(50K region)
>>
>> Other issue. Log is error:
>> see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9 should be see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A8
>>
>> Hmaster logs:
>>
>> 2011-05-06 19:31:09,924 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 2f19f33ae3f21ac4cb681f1662767d0c with OFFLINE state
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 76166ms instead of 60000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16697ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Forcing OFFLINE; was=ufdr,211007,1304669377398.696f124cc6ff82302f735c8413c6ac0b. state=CLOSED, ts=1304681364406
>> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 696f124cc6ff82302f735c8413c6ac0b with OFFLINE state
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Opened region ufdr,071415,1304668656420.aa026fbb27a25b0fe54039c00108dad6. on 157-5-100-9,20020,1304678135900
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for 7a75bac2028fba1529075225a3755c4c; deleting unassigned node
>> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for 7a75bac2028fba1529075225a3755c4c that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:31:38,986 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 12948ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:31:39,059 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Sent CLOSE to serverName=158-1-91-101,20020,1304663941215, load=(requests=0, regions=5374, usedHeap=79, maxHeap=8165) for region ufdr,082288,1304668716520.144efd9c886f5567abc9a229eb380c76.
>> 2011-05-06 19:31:39,059 INFO org.apache.hadoop.hbase.master.HMaster: balance hri=ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22., src=158-1-91-101,20020,1304663941215, dest=157-5-100-9,20020,1304678135900
>>
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-2011-05-06 19:31:54,741 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15681ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:11,419 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16574ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,176 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 11756ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:32:23,245 INFO org.apache.zookeeper.ClientCnxn: Unable to read additional data from server sessionid 0x12fc3a17c070054, likely server has closed socket, closing socket connection and attempting reconnect
>> 2011-05-06 19:32:36,902 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 13657ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,084 WARN org.apache.hadoop.hbase.zookeeper.ZKUtil: hconnection-0x12fc3a17c070054 Unable to set watcher on znode /hbase/root-region-server
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:35:44,096 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 78032ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,095 ERROR org.apache.hadoop.hbase.executor.EventHandler: Caught throwable while processing event RS_ZK_REGION_OPENED
>> java.lang.OutOfMemoryError: Java heap space
>>        at java.util.HashMap.addEntry(HashMap.java:753)
>>        at java.util.HashMap.put(HashMap.java:385)
>>        at java.util.HashMap.putAll(HashMap.java:524)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.updateTimers(AssignmentManager.java:630)
>>        at org.apache.hadoop.hbase.master.AssignmentManager.regionOnline(AssignmentManager.java:607)
>>        at org.apache.hadoop.hbase.master.handler.OpenedRegionHandler.process(OpenedRegionHandler.java:97)
>>        at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:151)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>        at java.lang.Thread.run(Thread.java:662)
>> 2011-05-06 19:35:59,981 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for accac1bddaa4bf102bcfa6506b19839d; deleting unassigned node
>> 2011-05-06 19:35:44,093 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Starting unassignment of region ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22. (offlining)
>> 2011-05-06 19:35:59,981 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
>> 2011-05-06 19:35:59,981 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15884ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:35:44,097 ERROR org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: hconnection-0x12fc3a17c070054 Received unexpected KeeperException, re-throwing exception
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,570 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16488ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
>> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-5-100-11/157.5.100.11:2181, initiating session
>> 2011-05-06 19:36:16,571 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 16489ms for sessionid 0x12fc3a17c070054, closing socket connection and attempting reconnect
>> 2011-05-06 19:36:00,082 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for accac1bddaa4bf102bcfa6506b19839d that is in expected state RS_ZK_REGION_OPENED
>> 2011-05-06 19:36:16,570 FATAL org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Unexpected exception during initialization, aborting
>> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>>        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:541)
>>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
>> 2011-05-06 19:36:16,572 INFO org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Reconnected successfully. This disconnect could have been caused by a network partition or a long-running GC pause, either way it's recommended that you verify your environment.
>> 2011-05-06 19:36:16,572 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down
>> 2011-05-06 19:36:16,572 WARN org.apache.hadoop.hbase.master.CatalogJanitor: Failed scan of catalog table
>> org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hbase.UnknownScannerException: Name: 3135625757230134514
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1794)
>>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1782)
>>        at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
>>        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>        at java.lang.reflect.Method.invoke(Method.java:597)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:570)
>>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039)
>>
>>        at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:771)
>>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
>>        at $Proxy6.next(Unknown Source)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:264)
>>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:237)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.scan(CatalogJanitor.java:116)
>>        at org.apache.hadoop.hbase.master.CatalogJanitor.chore(CatalogJanitor.java:85)
>>        at org.apache.hadoop.hbase.Chore.run(Chore.java:66)
>>
>

Re: Hmaster is OutOfMemory

Posted by Gaojinchao <ga...@huawei.com>.
Hbase version : 0.90.3RC0

It happened when creating table with Regions
I find master started needs so much memory when the cluster has 100K regions
It seems likes zkclientcnxn.

It seems master assigned region need improve.


top -c | grep 5834
5834 root      20   0 8875m 7.9g  11m S    2 50.5  33:53.19 /opt/jdk1.6.0_22/bin/java -Xmx8192m -ea -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode

Attaching to process ID 5834, please wait...
Debugger attached successfully.
Server compiler detected.
JVM version is 17.1-b03

using parallel threads in the new generation.
using thread-local object allocation.
Concurrent Mark-Sweep GC

Heap Configuration:
   MinHeapFreeRatio = 40
   MaxHeapFreeRatio = 70
   MaxHeapSize      = 8589934592 (8192.0MB)
   NewSize          = 21757952 (20.75MB)
   MaxNewSize       = 174456832 (166.375MB)
   OldSize          = 65404928 (62.375MB)
   NewRatio         = 7
   SurvivorRatio    = 8
   PermSize         = 21757952 (20.75MB)
   MaxPermSize      = 88080384 (84.0MB)

Heap Usage:
New Generation (Eden + 1 Survivor Space):
   capacity = 100335616 (95.6875MB)
   used     = 47094720 (44.91302490234375MB)
   free     = 53240896 (50.77447509765625MB)
   46.93719127612671% used
Eden Space:
   capacity = 89194496 (85.0625MB)
   used     = 35953600 (34.28802490234375MB)
   free     = 53240896 (50.77447509765625MB)
   40.30921369856723% used
From Space:
   capacity = 11141120 (10.625MB)
   used     = 11141120 (10.625MB)
   free     = 0 (0.0MB)
   100.0% used
To Space:
   capacity = 11141120 (10.625MB)
   used     = 0 (0.0MB)
   free     = 11141120 (10.625MB)
   0.0% used
concurrent mark-sweep generation:
   capacity = 8415477760 (8025.625MB)
   used     = 5107249280 (4870.6524658203125MB)
   free     = 3308228480 (3154.9725341796875MB)
   60.68876213155128% used
Perm Generation:
   capacity = 31199232 (29.75390625MB)
   used     = 18681784 (17.81633758544922MB)
   free     = 12517448 (11.937568664550781MB)
   59.87898676480241% used


-----邮件原件-----
发件人: jdcryans@gmail.com [mailto:jdcryans@gmail.com] 代表 Jean-Daniel Cryans
发送时间: 2011年5月10日 1:20
收件人: user@hbase.apache.org
主题: Re: Hmaster is OutOfMemory

It looks like the master entered a GC loop of death (since there are a
lot of "We slept 76166ms" messages) and finally died. Was it splitting
logs? Did you get a heap dump? Did you inspect it and can you tell
what was using all that space?

Thx,

J-D

2011/5/8 Gaojinchao <ga...@huawei.com>:
> Hbase version 0.90.2:
> Hmaster has 8G memory,  It seems like not enough ? why it needs so much memory?(50K region)
>
> Other issue. Log is error:
> see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9 should be see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A8
>
> Hmaster logs:
>
> 2011-05-06 19:31:09,924 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 2f19f33ae3f21ac4cb681f1662767d0c with OFFLINE state
> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 76166ms instead of 60000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16697ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Forcing OFFLINE; was=ufdr,211007,1304669377398.696f124cc6ff82302f735c8413c6ac0b. state=CLOSED, ts=1304681364406
> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 696f124cc6ff82302f735c8413c6ac0b with OFFLINE state
> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Opened region ufdr,071415,1304668656420.aa026fbb27a25b0fe54039c00108dad6. on 157-5-100-9,20020,1304678135900
> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for 7a75bac2028fba1529075225a3755c4c; deleting unassigned node
> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for 7a75bac2028fba1529075225a3755c4c that is in expected state RS_ZK_REGION_OPENED
> 2011-05-06 19:31:38,986 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 12948ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:31:39,059 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Sent CLOSE to serverName=158-1-91-101,20020,1304663941215, load=(requests=0, regions=5374, usedHeap=79, maxHeap=8165) for region ufdr,082288,1304668716520.144efd9c886f5567abc9a229eb380c76.
> 2011-05-06 19:31:39,059 INFO org.apache.hadoop.hbase.master.HMaster: balance hri=ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22., src=158-1-91-101,20020,1304663941215, dest=157-5-100-9,20020,1304678135900
>
> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-2011-05-06 19:31:54,741 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15681ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:32:11,419 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16574ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:32:23,176 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 11756ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:32:23,245 INFO org.apache.zookeeper.ClientCnxn: Unable to read additional data from server sessionid 0x12fc3a17c070054, likely server has closed socket, closing socket connection and attempting reconnect
> 2011-05-06 19:32:36,902 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 13657ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:35:44,084 WARN org.apache.hadoop.hbase.zookeeper.ZKUtil: hconnection-0x12fc3a17c070054 Unable to set watcher on znode /hbase/root-region-server
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>        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:541)
>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
> 2011-05-06 19:35:44,096 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 78032ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:35:44,095 ERROR org.apache.hadoop.hbase.executor.EventHandler: Caught throwable while processing event RS_ZK_REGION_OPENED
> java.lang.OutOfMemoryError: Java heap space
>        at java.util.HashMap.addEntry(HashMap.java:753)
>        at java.util.HashMap.put(HashMap.java:385)
>        at java.util.HashMap.putAll(HashMap.java:524)
>        at org.apache.hadoop.hbase.master.AssignmentManager.updateTimers(AssignmentManager.java:630)
>        at org.apache.hadoop.hbase.master.AssignmentManager.regionOnline(AssignmentManager.java:607)
>        at org.apache.hadoop.hbase.master.handler.OpenedRegionHandler.process(OpenedRegionHandler.java:97)
>        at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:151)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>        at java.lang.Thread.run(Thread.java:662)
> 2011-05-06 19:35:59,981 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for accac1bddaa4bf102bcfa6506b19839d; deleting unassigned node
> 2011-05-06 19:35:44,093 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Starting unassignment of region ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22. (offlining)
> 2011-05-06 19:35:59,981 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
> 2011-05-06 19:35:59,981 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15884ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:35:44,097 ERROR org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: hconnection-0x12fc3a17c070054 Received unexpected KeeperException, re-throwing exception
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>        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:541)
>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)5-100-11/157.5.100.11:2181, initiating session
> 2011-05-06 19:36:16,570 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16488ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-5-100-11/157.5.100.11:2181, initiating session
> 2011-05-06 19:36:16,571 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 16489ms for sessionid 0x12fc3a17c070054, closing socket connection and attempting reconnect
> 2011-05-06 19:36:00,082 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for accac1bddaa4bf102bcfa6506b19839d that is in expected state RS_ZK_REGION_OPENED
> 2011-05-06 19:36:16,570 FATAL org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Unexpected exception during initialization, aborting
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>        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:541)
>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
> 2011-05-06 19:36:16,572 INFO org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Reconnected successfully. This disconnect could have been caused by a network partition or a long-running GC pause, either way it's recommended that you verify your environment.
> 2011-05-06 19:36:16,572 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down
> 2011-05-06 19:36:16,572 WARN org.apache.hadoop.hbase.master.CatalogJanitor: Failed scan of catalog table
> org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hbase.UnknownScannerException: Name: 3135625757230134514
>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1794)
>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1782)
>        at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
>        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>        at java.lang.reflect.Method.invoke(Method.java:597)
>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:570)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039)
>
>        at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:771)
>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
>        at $Proxy6.next(Unknown Source)
>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:264)
>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:237)
>        at org.apache.hadoop.hbase.master.CatalogJanitor.scan(CatalogJanitor.java:116)
>        at org.apache.hadoop.hbase.master.CatalogJanitor.chore(CatalogJanitor.java:85)
>        at org.apache.hadoop.hbase.Chore.run(Chore.java:66)
>

Re: Hmaster is OutOfMemory

Posted by Jean-Daniel Cryans <jd...@apache.org>.
It looks like the master entered a GC loop of death (since there are a
lot of "We slept 76166ms" messages) and finally died. Was it splitting
logs? Did you get a heap dump? Did you inspect it and can you tell
what was using all that space?

Thx,

J-D

2011/5/8 Gaojinchao <ga...@huawei.com>:
> Hbase version 0.90.2:
> Hmaster has 8G memory,  It seems like not enough ? why it needs so much memory?(50K region)
>
> Other issue. Log is error:
> see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9 should be see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A8
>
> Hmaster logs:
>
> 2011-05-06 19:31:09,924 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 2f19f33ae3f21ac4cb681f1662767d0c with OFFLINE state
> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 76166ms instead of 60000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:31:09,924 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16697ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Forcing OFFLINE; was=ufdr,211007,1304669377398.696f124cc6ff82302f735c8413c6ac0b. state=CLOSED, ts=1304681364406
> 2011-05-06 19:31:09,932 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Creating (or updating) unassigned node for 696f124cc6ff82302f735c8413c6ac0b with OFFLINE state
> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Opened region ufdr,071415,1304668656420.aa026fbb27a25b0fe54039c00108dad6. on 157-5-100-9,20020,1304678135900
> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for 7a75bac2028fba1529075225a3755c4c; deleting unassigned node
> 2011-05-06 19:31:22,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for 7a75bac2028fba1529075225a3755c4c that is in expected state RS_ZK_REGION_OPENED
> 2011-05-06 19:31:38,986 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 12948ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:31:39,059 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Sent CLOSE to serverName=158-1-91-101,20020,1304663941215, load=(requests=0, regions=5374, usedHeap=79, maxHeap=8165) for region ufdr,082288,1304668716520.144efd9c886f5567abc9a229eb380c76.
> 2011-05-06 19:31:39,059 INFO org.apache.hadoop.hbase.master.HMaster: balance hri=ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22., src=158-1-91-101,20020,1304663941215, dest=157-5-100-9,20020,1304678135900
>
> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-2011-05-06 19:31:54,741 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15681ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:32:11,419 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16574ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:32:23,176 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 11756ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:32:23,245 INFO org.apache.zookeeper.ClientCnxn: Unable to read additional data from server sessionid 0x12fc3a17c070054, likely server has closed socket, closing socket connection and attempting reconnect
> 2011-05-06 19:32:36,902 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 13657ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:35:44,084 WARN org.apache.hadoop.hbase.zookeeper.ZKUtil: hconnection-0x12fc3a17c070054 Unable to set watcher on znode /hbase/root-region-server
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>        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:541)
>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
> 2011-05-06 19:35:44,096 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 78032ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:35:44,095 ERROR org.apache.hadoop.hbase.executor.EventHandler: Caught throwable while processing event RS_ZK_REGION_OPENED
> java.lang.OutOfMemoryError: Java heap space
>        at java.util.HashMap.addEntry(HashMap.java:753)
>        at java.util.HashMap.put(HashMap.java:385)
>        at java.util.HashMap.putAll(HashMap.java:524)
>        at org.apache.hadoop.hbase.master.AssignmentManager.updateTimers(AssignmentManager.java:630)
>        at org.apache.hadoop.hbase.master.AssignmentManager.regionOnline(AssignmentManager.java:607)
>        at org.apache.hadoop.hbase.master.handler.OpenedRegionHandler.process(OpenedRegionHandler.java:97)
>        at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:151)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>        at java.lang.Thread.run(Thread.java:662)
> 2011-05-06 19:35:59,981 DEBUG org.apache.hadoop.hbase.master.handler.OpenedRegionHandler: Handling OPENED event for accac1bddaa4bf102bcfa6506b19839d; deleting unassigned node
> 2011-05-06 19:35:44,093 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Starting unassignment of region ufdr,050142,1304668531330.2d56755e7138a93b243a7c5363d1ac22. (offlining)
> 2011-05-06 19:35:59,981 INFO org.apache.zookeeper.ClientCnxn: Opening socket connection to server /157.5.100.11:2181
> 2011-05-06 19:35:59,981 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 15884ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:35:44,097 ERROR org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: hconnection-0x12fc3a17c070054 Received unexpected KeeperException, re-throwing exception
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>        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:541)
>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)5-100-11/157.5.100.11:2181, initiating session
> 2011-05-06 19:36:16,570 WARN org.apache.hadoop.hbase.util.Sleeper: We slept 16488ms instead of 1000ms, this is likely due to a long garbage collecting pause and it's usually bad, see http://wiki.apache.org/hadoop/Hbase/Troubleshooting#A9
> 2011-05-06 19:36:16,570 INFO org.apache.zookeeper.ClientCnxn: Socket connection established to 157-5-100-11/157.5.100.11:2181, initiating session
> 2011-05-06 19:36:16,571 INFO org.apache.zookeeper.ClientCnxn: Client session timed out, have not heard from server in 16489ms for sessionid 0x12fc3a17c070054, closing socket connection and attempting reconnect
> 2011-05-06 19:36:00,082 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:20000-0x12fc3a17c070022 Deleting existing unassigned node for accac1bddaa4bf102bcfa6506b19839d that is in expected state RS_ZK_REGION_OPENED
> 2011-05-06 19:36:16,570 FATAL org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Unexpected exception during initialization, aborting
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/root-region-server
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
>        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:819)
>        at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:232)
>        at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:72)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.setupZookeeperTrackers(HConnectionManager.java:308)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.resetZooKeeperTrackers(HConnectionManager.java:319)
>        at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1333)
>        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:541)
>        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:517)
> 2011-05-06 19:36:16,572 INFO org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation: Reconnected successfully. This disconnect could have been caused by a network partition or a long-running GC pause, either way it's recommended that you verify your environment.
> 2011-05-06 19:36:16,572 INFO org.apache.zookeeper.ClientCnxn: EventThread shut down
> 2011-05-06 19:36:16,572 WARN org.apache.hadoop.hbase.master.CatalogJanitor: Failed scan of catalog table
> org.apache.hadoop.ipc.RemoteException: org.apache.hadoop.hbase.UnknownScannerException: Name: 3135625757230134514
>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1794)
>        at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1782)
>        at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source)
>        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>        at java.lang.reflect.Method.invoke(Method.java:597)
>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:570)
>        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039)
>
>        at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:771)
>        at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
>        at $Proxy6.next(Unknown Source)
>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:264)
>        at org.apache.hadoop.hbase.catalog.MetaReader.fullScan(MetaReader.java:237)
>        at org.apache.hadoop.hbase.master.CatalogJanitor.scan(CatalogJanitor.java:116)
>        at org.apache.hadoop.hbase.master.CatalogJanitor.chore(CatalogJanitor.java:85)
>        at org.apache.hadoop.hbase.Chore.run(Chore.java:66)
>