[ 
https://issues.apache.org/jira/browse/HBASE-3323?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12971970#action_12971970
 ] 

stack commented on HBASE-3323:
------------------------------

I did some more testing and came across the following after split was done and 
HLogSplitter was moving to close all files:

{code}
2010-12-16 01:07:20,394 INFO 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Waiting for split writer 
threads to finish
2010-12-16 01:07:20,640 INFO 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Split writers finished
2010-12-16 01:07:20,650 ERROR 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Couldn't close log at 
hdfs://sv2borg180:10000/hbase/TestTable/02495f8b7cb6404cb7ea0521cc183d56/recovered.edits/0000000000000012854
org.apache.hadoop.ipc.RemoteException: 
org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on 
/hbase/TestTable/02495f8b7cb6404cb7ea0521cc183d56/recovered.edits/0000000000000012854
 File does not exist. [Lease.  Holder: DFSClient_hb_m_sv2borg180:         
60000_1292460569453, pendingcreates: 178]
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:1418)
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:1409)
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:1464)
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:1452)
    at 
org.apache.hadoop.hdfs.server.namenode.NameNode.complete(NameNode.java:471)
    at sun.reflect.GeneratedMethodAccessor68.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:508)
    at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:961)
    at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:957)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:396)
    at org.apache.hadoop.ipc.Server$Handler.run(Server.java:955)
    at org.apache.hadoop.ipc.Client.call(Client.java:740)
    at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:220)
    at $Proxy5.complete(Unknown Source)
    at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
    at java.lang.reflect.Method.invoke(Method.java:597)
    at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
    at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
    at $Proxy5.complete(Unknown Source)
    at 
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3457)
    at 
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3381)
    at 
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:61)
    at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:86)
    at org.apache.hadoop.io.SequenceFile$Writer.close(SequenceFile.java:966)
    at 
org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.close(SequenceFileLogWriter.java:138)
    at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.closeStreams(HLogSplitter.java:756)
    at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter$OutputSink.finishWritingAndClose(HLogSplitter.java:741)
    at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLog(HLogSplitter.java:291)
    at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLog(HLogSplitter.java:186)
    at 
org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:194)
    at 
org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.process(ServerShutdownHandler.java:96)
    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)
2010-12-16 01:07:20,685 INFO 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Closed path 
hdfs://sv2borg180:10000/hbase/TestTable/03dbd921c75876d6bc3f86c10201fa93/recovered.edits/0000000000000014196
 (wrote 12 edits in 470ms)
2010-12-16 01:07:20,719 INFO 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Closed path 
hdfs://sv2borg180:10000/hbase/TestTable/04f81e343d032d43946393636b2b4d2d/recovered.edits/0000000000000012928
 (wrote 15 edits in 390ms)
2010-12-16 01:07:20,725 ERROR 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Couldn't close log at 
hdfs://sv2borg180:10000/hbase/TestTable/06d137bd176e2604761243d396c11b3a/recovered.edits/0000000000000012945
org.apache.hadoop.ipc.RemoteException: 
org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException: No lease on 
/hbase/TestTable/06d137bd176e2604761243d396c11b3a/recovered.edits/0000000000000012945
 File does not exist. [Lease.  Holder: DFSClient_hb_m_sv2borg180:         
60000_1292460569453, pendingcreates: 176]
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:1418)
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:1409)
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:1464)
    at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:1452)
...
{code}

It then closes a bunch and fails one or two more until its done with all.

Eventually the split 'completes' and we start assigning out regions:

{code}
2010-12-16 01:07:32,581 INFO 
org.apache.hadoop.hbase.master.handler.ServerShutdownHandler: Reassigning 186 
region(s) that sv2borg185,60020,1292460570976 was carrying (skipping 1 
regions(s) that are already in transition)
{code}



> OOME in master splitting logs
> -----------------------------
>
>                 Key: HBASE-3323
>                 URL: https://issues.apache.org/jira/browse/HBASE-3323
>             Project: HBase
>          Issue Type: Bug
>          Components: master
>    Affects Versions: 0.90.0
>            Reporter: Todd Lipcon
>            Assignee: Todd Lipcon
>            Priority: Blocker
>             Fix For: 0.90.0
>
>         Attachments: hbase-3323.4.txt, hbase-3323.5.txt, hbase-3323.txt, 
> hbase-3323.txt, hbase-3323.txt, sizes.png
>
>
> In testing a RS failure under heavy increment workload I ran into an OOME 
> when the master was splitting the logs.
> In this test case, I have exactly 136 bytes per log entry in all the logs, 
> and the logs are all around 66-74MB). With a batch size of 3 logs, this means 
> the master is loading about 500K-600K edits per log file. Each edit ends up 
> creating 3 byte[] objects, the references for which are each 8 bytes of RAM, 
> so we have 160 (136+8*3) bytes per edit used by the byte[]. For each edit we 
> also allocate a bunch of other objects: one HLog$Entry, one WALEdit, one 
> ArrayList, one LinkedList$Entry, one HLogKey, and one KeyValue. Overall this 
> works out to 400 bytes of overhead per edit. So, with the default settings on 
> this fairly average workload, the 1.5M log entries takes about 770MB of RAM. 
> Since I had a few log files that were a bit larger (around 90MB) it exceeded 
> 1GB of RAM and I got an OOME.
> For one, the 400 bytes per edit overhead is pretty bad, and we could probably 
> be a lot more efficient. For two, we should actually account this rather than 
> simply having a configurable "batch size" in the master.
> I think this is a blocker because I'm running with fairly default configs 
> here and just killing one RS made the cluster fall over due to master OOME.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to