[ https://issues.apache.org/jira/browse/HBASE-5077?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13173575#comment-13173575 ]
Jean-Daniel Cryans edited comment on HBASE-5077 at 12/20/11 10:35 PM: ---------------------------------------------------------------------- One problem I see is that in HLogSplitter.splitLogFileToTemp we do this in the finally: {code} if ((progress_failed == false) && (reporter != null) && (reporter.progress() == false)) { progress_failed = true; } {code} But at this point progress_failed isn't taken into account so the method returns true. Looking at other parts of that method it seems it's missing a "return false" which would be correctly handled by SplitLogWorker. was (Author: jdcryans): One problem I see is that in HLogSplitter.splitLogFileToTemp we do this in the finally: {quote} if ((progress_failed == false) && (reporter != null) && (reporter.progress() == false)) { progress_failed = true; } {quote} But at this point progress_failed isn't taken into account so the method returns true. Looking at other parts of that method it seems it's missing a "return false" which would be correctly handled by SplitLogWorker. > 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