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

chunhui shen commented on HBASE-6309:
-------------------------------------

[~devaraj]
bq.I guess its not as bad as it used to be (the factor of #renames reduced from 
#regions to #logs)
Yes, it's so. IMO, we don't need to fix this issue because now there is only 
one NN operation inside of the ZK EventThread in SplitLogManager.

What's the bottleneck of current distributed-log-split?
If the whole cluster or many servers are down, there are lots of NN operations 
because each hlog file will split into many RecoveredEdits files. 
However,local-log-split only generate one RecoveredEdits file in one region for 
the whole server hlogs directory
If we can improve the task assign mechanism on SplitLogManager or task grab 
mechanism on SplitLogWorker, I think it's great. e.g. SplitLogWorker could 
split many hlog files in one time




                
> [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager
> ------------------------------------------------------------------------
>
>                 Key: HBASE-6309
>                 URL: https://issues.apache.org/jira/browse/HBASE-6309
>             Project: HBase
>          Issue Type: Improvement
>    Affects Versions: 0.92.1, 0.94.0, 0.96.0
>            Reporter: Jean-Daniel Cryans
>            Assignee: Devaraj Das
>             Fix For: 0.96.0
>
>
> We found this issue during the leap second cataclysm which prompted a 
> distributed splitting of all our logs.
> I saw that none of the RS were splitting after some time while the master was 
> showing that it wasn't even 30% done. jstack'ing I saw this:
> {noformat}
> "main-EventThread" daemon prio=10 tid=0x00007f6ce46d8800 nid=0x5376 in
> Object.wait() [0x00007f6ce2ecb000]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1093)
>         - locked <0x00000005fdd661a0> (a org.apache.hadoop.ipc.Client$Call)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:226)
>         at $Proxy9.rename(Unknown Source)
>         at sun.reflect.GeneratedMethodAccessor29.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 $Proxy9.rename(Unknown Source)
>         at org.apache.hadoop.hdfs.DFSClient.rename(DFSClient.java:759)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem.rename(DistributedFileSystem.java:253)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.moveRecoveredEditsFromTemp(HLogSplitter.java:553)
>         at 
> org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.moveRecoveredEditsFromTemp(HLogSplitter.java:519)
>         at 
> org.apache.hadoop.hbase.master.SplitLogManager$1.finish(SplitLogManager.java:138)
>         at 
> org.apache.hadoop.hbase.master.SplitLogManager.getDataSetWatchSuccess(SplitLogManager.java:431)
>         at 
> org.apache.hadoop.hbase.master.SplitLogManager.access$1200(SplitLogManager.java:95)
>         at 
> org.apache.hadoop.hbase.master.SplitLogManager$GetDataAsyncCallback.processResult(SplitLogManager.java:1011)
>         at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:571)
>         at 
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:497)
> {noformat}
> We are effectively bottlenecking on doing NN operations and whatever else is 
> happening in GetDataAsyncCallback. It was so bad that on our 100 offline 
> cluster it took a few hours for the master to process all the incoming ZK 
> events while the actual splitting took a fraction of that time.
> I'm marking this as critical and against 0.96 but depending on how involved 
> the fix is we might want to backport.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to