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

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

This is from the master's POV:

{quote}
2011-12-20 02:59:42,086 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: 
put up splitlog task at znode 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
2011-12-20 02:59:42,089 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: 
task not yet acquired 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 ver = 0
2011-12-20 02:59:42,113 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 acquired by sv4r13s38,62023,1324345934996
2011-12-20 03:00:09,244 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
resubmitting task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
2011-12-20 03:00:09,302 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: 
task not yet acquired 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 ver = 3
2011-12-20 03:02:53,072 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 acquired by sv4r28s44,62023,1324345934970
2011-12-20 03:03:21,117 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
resubmitting task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
2011-12-20 03:03:21,136 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: 
task not yet acquired 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 ver = 6
2011-12-20 03:04:40,421 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 acquired by sv4r6s38,62023,1324345935082
2011-12-20 03:05:09,133 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
resubmitting task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
2011-12-20 03:05:09,144 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: 
task not yet acquired 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 ver = 9
2011-12-20 03:05:09,193 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 acquired by sv4r30s44,62023,1324345935039
2011-12-20 03:05:36,137 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
Skipping resubmissions of task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 because threshold 3 reached
...
2011-12-20 03:05:47,139 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
Skipping resubmissions of task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 because threshold 3 reached
2011-12-20 03:05:50,320 INFO org.apache.hadoop.hbase.master.SplitLogManager: 
task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 entered state done sv4r30s44,62023,1324345935039
{quote}

The one that died is sv4r6s38, the 3rd one to acquire the task. Here's its log:

{quote}
2011-12-20 03:04:40,418 INFO 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: worker 
sv4r6s38,62023,1324345935082 acquired task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
2011-12-20 03:04:43,574 DEBUG 
org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter: 
Path=hdfs://sv4r11s38:9100/hbase/splitlog/sv4r6s38,62023,1324345935082_hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814/TestTable/18010aed747eeb11d1a95427731bb136/recovered.edits/0000000000000031373.temp,
 syncFs=true, hflush=false
...
{quote}

Then it sees someone else takes the task but moves on:

{quote}
2011-12-20 03:05:09,182 INFO 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 preempted from sv4r6s38,62023,1324345935082, current task state and 
owner=unassigned sv4r11s38,62003,1324340331847
2011-12-20 03:05:09,194 INFO 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 preempted from sv4r6s38,62023,1324345935082, current task state and 
owner=owned sv4r30s44,62023,1324345935039
2011-12-20 03:05:10,841 INFO 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: task 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
 preempted from sv4r6s38,62023,1324345935082, current task state and 
owner=owned sv4r30s44,62023,1324345935039
2011-12-20 03:05:15,089 DEBUG 
org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter: 
Path=hdfs://sv4r11s38:9100/hbase/splitlog/sv4r6s38,62023,1324345935082_hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814/TestTable/e710a55d352791b42ea6d588ed38e934/recovered.edits/0000000000000031401.temp,
 syncFs=true, hflush=false
{quote}

This happens another time until it fails to assert ownership of the znode:

{quote}
2011-12-20 03:06:17,256 WARN 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: NONODE failed to assert 
ownership for 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for 
/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
2011-12-20 03:06:17,256 WARN 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: Failed to heartbeat the 
task/hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
2011-12-20 03:06:17,302 DEBUG 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Closed 
hdfs://sv4r11s38:9100/hbase/splitlog/sv4r6s38,62023,1324345935082_hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814/TestTable/12c66fe6163486f5d1d1b6a2b7a5a474/recovered.edits/0000000000000031403.temp
...
{quote}

Like a honey badger it doesn't really care and continues closing files until it 
reaches a point where it dies with the stack posted in this jira's description.
                
> SplitLogWorker fails to let go of a task, kills the RS
> ------------------------------------------------------
>
>                 Key: HBASE-5077
>                 URL: https://issues.apache.org/jira/browse/HBASE-5077
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.92.0
>            Reporter: Jean-Daniel Cryans
>            Priority: Critical
>             Fix For: 0.92.1
>
>
> I hope I didn't break spacetime continuum, I got this while testing 0.92.0:
> {quote}
> 2011-12-20 03:06:19,838 FATAL 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker: logic error - end task 
> /hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
>  done failed because task doesn't exist
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for 
> /hbase/splitlog/hdfs%3A%2F%2Fsv4r11s38%3A9100%2Fhbase%2F.logs%2Fsv4r14s38%2C62023%2C1324345935047-splitting%2Fsv4r14s38%252C62023%252C1324345935047.1324349363814
>         at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
>         at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>         at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1228)
>         at 
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:372)
>         at org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:654)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.endTask(SplitLogWorker.java:372)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.grabTask(SplitLogWorker.java:280)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.taskLoop(SplitLogWorker.java:197)
>         at 
> org.apache.hadoop.hbase.regionserver.SplitLogWorker.run(SplitLogWorker.java:165)
>         at java.lang.Thread.run(Thread.java:662)
> {quote}
> I'll post more logs in a moment. What I can see is that the master shuffled 
> that task around a bit and one of the region servers died on this stack trace 
> while the others were able to interrupt themselves.

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

        

Reply via email to