[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-15 Thread Devaraj Das (JIRA)

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

Devaraj Das commented on HBASE-6309:


So yeah the majority of the FS operations now is in the 
HLogSplitter.finishSplitLogFile (and in the archiveLogs it calls internally). I 
guess its not as bad as it used to be (the factor of #renames reduced from 
#regions to #logs). Correct me if I am wrong, [~zjushch].

On the invariant, yes, we probably should avoid storing the queue in ZK (and 
just have in memory queue). The master should rebuild the queue after it comes 
up by making a pass over the logdirs.

So what do folks think? Should we move the FS operations out of the critical 
path in SplitLogManager's event handler? Or leave this fix to some other day?

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-15 Thread stack (JIRA)

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

stack commented on HBASE-6309:
--

[~devaraj] Sounds like, given the above, this is no longer a critical issue.  
What say you boss?  We could mark it major?

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-15 Thread chunhui shen (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-6309?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-14 Thread Devaraj Das (JIRA)

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

Devaraj Das commented on HBASE-6309:


How about we queue up the call [finishSplitLogFile + params] in a ZK queue. The 
master could have a thread that removes finishSplitLogFile calls off this queue 
and execute them (on a master failure, the queue will still have the pending 
calls intact). There might be issues with this approach but just wanted to 
throw it out there while I do more detailed study / prototype.. 

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-14 Thread chunhui shen (JIRA)

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

chunhui shen commented on HBASE-6309:
-

I think we don't need do this now since HBASE-6337 (NN operations are not done 
in SplitLogManager any more)

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-14 Thread stack (JIRA)

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

stack commented on HBASE-6309:
--

[~devaraj] So, when callback is called, you'd stick the even up in zk queue?  
Then in another thread process it?  Queuing an executor to do the NN work would 
be less than this.  A queue would be good in case server died, yes.  I wonder 
what else we should be queuing around split log trans?  Would having a Q 
violate 15.10.4 Invariant: http://hbase.apache.org/book.html#developing?

[~zjushch] That is interesting.  If so, we should be able to just close this?  
What you reckon DD?

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-14 Thread chunhui shen (JIRA)

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

chunhui shen commented on HBASE-6309:
-

bq.If so, we should be able to just close this?
Yes
Before HBASE-6337, SplitLogManager will move lots of RecoveredEdits from temp 
when finishing a split log task.
But now we needn't these movement because RecoveredEdits are created in the 
region dir by SplitLogWorker, not in a temp dir.

[~devaraj]
Could you confirm it?
Thanks

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2013-01-14 Thread Devaraj Das (JIRA)

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

Devaraj Das commented on HBASE-6309:


Okay, Stack and Chunhui. Will update by tomorrow.

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-09-25 Thread nkeywal (JIRA)

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

nkeywal commented on HBASE-6309:


I'm was having a look at this. Could we have the log archiving done by the 
regionserver instead of the master? This would lower the work done in the event 
thread? The only remaining stuff would be the renaming of the region log dir at 
the end. 

I see one impact: if the same log was processed simultaneously by multiple 
region server, this archiving could occur in parallel on two different region 
server. Manageable I think...

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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


[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-04 Thread stack (JIRA)

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

stack commented on HBASE-6309:
--

@Chunhui What about case where we fail a log splitting... how would the cleanup 
go?  If into a tmp dir, its easy remove the tmp dir (Otherwise, sounds like a 
fine idea).

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-04 Thread chunhui shen (JIRA)

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

chunhui shen commented on HBASE-6309:
-

bq.how would the cleanup go?
In HLogSplitter#createWAP
{code}
if ((tmpname == null)  fs.exists(regionedits)) {
  LOG.warn(Found existing old edits file. It could be the 
  + result of a previous failed split attempt. Deleting 
  + regionedits + , length=
  + fs.getFileStatus(regionedits).getLen());
  if (!fs.delete(regionedits, false)) {
LOG.warn(Failed delete of old  + regionedits);
  }
}
{code}
We could also fail a log splitting if using master-local-splitting, the clean 
up happen in the next splitting as per the above code

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-04 Thread stack (JIRA)

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

stack commented on HBASE-6309:
--

What about the logic in moveRecoveredEditsFromTemp?  It flags corrupted logs 
and does some other cleanup.  Also seems to find recovered.edits files with a 
.corrupt ending: see ZKSlitLog.isCorruptFlagFile That'd need refactoring 
and a rename from moveRecoveredEditsFromTemp to 'completeLogSplit' or 'finish'?

Otherwise, looking through HLogSplitting and trying to recall issues we've run 
into w/ recovered.edits, I think doing it in place can work.

Would suggest you look at the region open and replay of recovered.edits stuff 
too to see if you see any possible issues there (I only went through 
HLogSplitting).

(That renaming stuff is pretty heavy duty stuffbut I'd have done the same 
to cordon off a distributed operation)

Good stuff Chunhui.

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-04 Thread ramkrishna.s.vasudevan (JIRA)

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

ramkrishna.s.vasudevan commented on HBASE-6309:
---

Currently there are 3 renames in this path.
The one that renames the temp to recovered.edits path and the next is in 
archive logs. Here there are 2 one for corrupted ones and the other for 
archived path.

In between there are lot of deletes and exists call.  I think we can reduce no 
of NN operations.  How costly is delete and exists check? I will check on this 
more.


 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-04 Thread nkeywal (JIRA)

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

nkeywal commented on HBASE-6309:


bq. How costly is delete and exists check?
A remote call to the NN, but no socket creation (it's persistent). There is no 
cache on the client side, so all exists calls will do the network loop. Exists 
is pretty fast (not much more cost than the network roundtrip), but it adds a 
little something to the NN and network workload that can be already high when 
there is a major failure...

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-03 Thread ramkrishna.s.vasudevan (JIRA)

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

ramkrishna.s.vasudevan commented on HBASE-6309:
---

@JD
HBASE-6140 is same as this?

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-03 Thread Jean-Daniel Cryans (JIRA)

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

Jean-Daniel Cryans commented on HBASE-6309:
---

Hey Ram, yeah it's pretty much the same although I like my description of the 
problem better :)

IMO we should move everything that talks to ZK and NN out of that path.

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-03 Thread nkeywal (JIRA)

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

nkeywal commented on HBASE-6309:


bq. IMO we should move everything that talks to ZK and NN out of that path.
+1...

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-6309) [MTTR] Do NN operations outside of the ZK EventThread in SplitLogManager

2012-07-03 Thread chunhui shen (JIRA)

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

chunhui shen commented on HBASE-6309:
-

In current distributed-log-splitting, we will split the logs to a tmp dir.
How about we directly splitting logs to region dir, so no necessary to do NN 
operations in finishing task for SplitLogManager.

 [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
Priority: Critical
 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=0x7f6ce46d8800 nid=0x5376 in
 Object.wait() [0x7f6ce2ecb000]
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 0x0005fdd661a0 (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: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira