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

Jonathan Hsieh commented on HBASE-7703:
---------------------------------------

This is actually caused buy the flush snapshot attemptig to take a region lock 
and seemingly getting stuck timing out here:

{code}
2013-01-28 13:24:00,681 WARN org.apache.hadoop.hbase.regionserver.HRegion: 
Failed getting lock in batch put, row=0001558252
java.io.IOException: Timed out on getting lock for row=0001558252
        at 
org.apache.hadoop.hbase.regionserver.HRegion.internalObtainRowLock(HRegion.java:3239)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.getLock(HRegion.java:3315)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2150)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2021)
        at 
org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3511)
        at sun.reflect.GeneratedMethodAccessor46.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.WritableRpcEngine$Server.call(WritableRpcEngine.java:364)
        at 
org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1400)
2013-01-28 13:24:00,682 DEBUG org.apache.hadoop.hbase.regionserver.HRegion: 
rollbackMemstore rolled back 0 keyvalues from start:0 to end:0
2013-01-28 13:24:04,715 WARN org.apache.hadoop.hbase.regionserver.HRegion: 
Failed getting lock in batch put, row=0001559792
java.io.IOException: Timed out on getting lock for row=0001559792
        at 
org.apache.hadoop.hbase.regionserver.HRegion.internalObtainRowLock(HRegion.java:3239)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.getLock(HRegion.java:3315)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2150)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2021)
        at 
org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3511)
        at sun.reflect.GeneratedMethodAccessor46.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.WritableRpcEngine$Server.call(WritableRpcEngine.java:364)
        at 
org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1400)
{code}

It seems the root cause of this is actually a previously failed snapshot 
attempt that grabs the rowlock and likely does not release it properly.  

{code}
2013-01-26 01:54:56,417 ERROR 
org.apache.hadoop.hbase.procedure.ProcedureMember: Propagating foreign 
exception to subprocedure pe-1
org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable via 
timer-java.util.Timer@1cea3151:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable:
 org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! 
Source:Timeout caused Foreign E
xception Start:1359194035004, End:1359194095004, diff:60000, max:60000 ms
        at 
org.apache.hadoop.hbase.errorhandling.ForeignException.deserialize(ForeignException.java:184)
        at 
org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.abort(ZKProcedureMemberRpcs.java:321)
        at 
org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.watchForAbortedProcedures(ZKProcedureMemberRpcs.java:150)
        at 
org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.access$200(ZKProcedureMemberRpcs.java:56)
        at 
org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs$1.nodeChildrenChanged(ZKProcedureMemberRpcs.java:112)
        at 
org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:315)
        at 
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:519)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:495)
Caused by: 
org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: 
org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! 
Source:Timeout caused Foreign Exception Start:1359194035004, End:1359194095004, 
diff:60000, max:60000 ms
        at 
org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71)
        at java.util.TimerThread.mainLoop(Timer.java:512)
        at java.util.TimerThread.run(Timer.java:462)
2013-01-26 01:54:56,648 WARN org.apache.hadoop.hbase.regionserver.HRegion: 
Failed getting lock in batch put, row=0001558252
java.io.IOException: Timed out on getting lock for row=0001558252
        at 
org.apache.hadoop.hbase.regionserver.HRegion.internalObtainRowLock(HRegion.java:3239)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.getLock(HRegion.java:3315)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2150)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2021)
        at 
org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3511)
        at sun.reflect.GeneratedMethodAccessor46.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.WritableRpcEngine$Server.call(WritableRpcEngine.java:364)
        at 
org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1400)
{code}

Currently, if we abort we cancel region specific tasks with thread 
Interruptions.  I believe we likely have a problem in HRegion with individual 
rowlocks when an interruption occurs Restarting the RS seems to allow for 
snapshots to be taken againt.  I believe the rowlock not released, and since 
this lock is in memory it sticks around until rs rebooted.  

The quick fix I'm currently testing is to not allow thread interrupts when 
cancelling -- instead we wait for inflight tasks to complete naturally and 
block those that have not started from starting.
                
> Eventually all online snapshots failing due to Timeout at same regionserver.
> ----------------------------------------------------------------------------
>
>                 Key: HBASE-7703
>                 URL: https://issues.apache.org/jira/browse/HBASE-7703
>             Project: HBase
>          Issue Type: Sub-task
>          Components: snapshots
>    Affects Versions: hbase-7290
>            Reporter: Jonathan Hsieh
>            Assignee: Jonathan Hsieh
>            Priority: Critical
>             Fix For: hbase-7290
>
>
> After running for a long time, we eventually get to a point where we get this 
> exception for all snapshot attempts:
> {code}
> 2013-01-27 23:51:36,380 ERROR org.apache.hadoop.hbase.procedure.Procedure: 
> Procedure 'pe-15' execution failed!
> org.apache.hadoop.hbase.errorhandling.TimeoutException via 
> timer-java.util.Timer@35e7d702:org.apache.hadoop.hbase.errorhandling.TimeoutException:
>  Timeout elapsed! Source:Timeout caused Foreign Exception 
> Start:1359359435967, End:1359359495967, diff:60000, max:60000 ms
>         at 
> org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:84)
>         at 
> org.apache.hadoop.hbase.procedure.Procedure.waitForLatch(Procedure.java:357)
>         at 
> org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:196)
>         at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:68)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>         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)
> Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout 
> elapsed! Source:Timeout caused Foreign Exception Start:1359359435967, 
> End:1359359495967, diff:60000, max:60000 ms
>         at 
> org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
>         at java.util.TimerThread.mainLoop(Timer.java:512)
>         at java.util.TimerThread.run(Timer.java:462)
> {code}

--
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