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

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

I now understand how I got all the way to closing the files without aborting 
the splitting, the interrupt is being retried by the DFSClient:

{quote}

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:09,194 INFO 
org.apache.hadoop.hbase.regionserver.SplitLogWorker: Sending interrupt to stop 
the worker thread
2011-12-20 03:05:09,214 INFO org.apache.hadoop.hdfs.DFSClient: Failed to 
connect to /10.4.28.44:51010, add to deadNodes and continue
java.nio.channels.ClosedByInterruptException
        at 
java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:184)
        at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:511)
        at 
org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:192)
        at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:408)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.getBlockReader(DFSClient.java:2354)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.blockSeekTo(DFSClient.java:2033)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.seekToBlockSource(DFSClient.java:2483)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:2119)
        at 
org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:2150)
        at java.io.DataInputStream.read(DataInputStream.java:132)
        at java.io.DataInputStream.readFully(DataInputStream.java:178)
        at 
org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:63)
        at 
org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:101)
        at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1945)
        at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1845)
        at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1891)
        at 
org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.next(SequenceFileLogReader.java:198)
        at 
org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogReader.next(SequenceFileLogReader.java:172)
        at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.getNextLogLine(HLogSplitter.java:764)
        at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:402)
        at 
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.splitLogFileToTemp(HLogSplitter.java:351)
        at 
org.apache.hadoop.hbase.regionserver.SplitLogWorker$1.exec(SplitLogWorker.java:113)
        at 
org.apache.hadoop.hbase.regionserver.SplitLogWorker.grabTask(SplitLogWorker.java:266)
        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)
2011-12-20 03:05:09,216 INFO org.apache.hadoop.hdfs.DFSClient: Failed to 
connect to /10.4.12.38:51010, add to deadNodes and continue
java.nio.channels.ClosedByInterruptException
        at 
java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:184)
...
2011-12-20 03:05:09,220 INFO org.apache.hadoop.hdfs.DFSClient: Failed to 
connect to /10.4.14.38:51010, add to deadNodes and continue
java.nio.channels.ClosedByInterruptException
        at 
java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:184)
...
2011-12-20 03:05:09,223 INFO org.apache.hadoop.hdfs.DFSClient: Could not obtain 
block blk_2118163224139708562_43382 from any node: java.io.IOException: No live 
nodes contain current block. Will get new block locations from namenode and 
retry...
{quote}
                
> 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