[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-21 Thread Hudson (Commented) (JIRA)

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

Hudson commented on HBASE-5077:
---

Integrated in HBase-0.92-security #47 (See 
[https://builds.apache.org/job/HBase-0.92-security/47/])
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS -- fix 
compile error
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS

stack : 
Files : 
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java

stack : 
Files : 
* /hbase/branches/0.92/CHANGES.txt
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java


 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.0

 Attachments: HBASE-5077-v2.patch, HBASE-5077-v4.txt, HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-21 Thread Hudson (Commented) (JIRA)

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

Hudson commented on HBASE-5077:
---

Integrated in HBase-TRUNK-security #39 (See 
[https://builds.apache.org/job/HBase-TRUNK-security/39/])
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS -- fix 
compile error
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS

stack : 
Files : 
* 
/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* 
/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java


 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.0

 Attachments: HBASE-5077-v2.patch, HBASE-5077-v4.txt, HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Jean-Daniel Cryans (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5077?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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: 

[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Zhihong Yu (Commented) (JIRA)

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

Zhihong Yu commented on HBASE-5077:
---

After preemption log, the following code should have run:
{code}
  void stopTask() {
LOG.info(Sending interrupt to stop the worker thread);
worker.interrupt(); // TODO interrupt often gets swallowed, do what else?
  }
{code}
I think the following method should have been called instead:
{code}
  public void stop() {
exitWorker = true;
stopTask();
  }
{code}

 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Jean-Daniel Cryans (Commented) (JIRA)

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

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

Won't exitWorker kill the SplitLogWorker fully? Like not just the task, but the 
RS will actually stop serving log splitting.

 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Jean-Daniel Cryans (Commented) (JIRA)

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

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

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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Zhihong Yu (Commented) (JIRA)

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

Zhihong Yu commented on HBASE-5077:
---

To answer J-D's question, let me reference the following code from taskLoop():
{code}
  } catch (InterruptedException e) {
LOG.info(SplitLogWorker interrupted while waiting for task, +
   exiting:  + e.toString());
assert exitWorker == true;
return;
  }
{code}
where exitWorker was expected to be true. I think the assertion wasn't 
triggered at runtime.

 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Jean-Daniel Cryans (Commented) (JIRA)

[ 
https://issues.apache.org/jira/browse/HBASE-5077?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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 
 

[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Zhihong Yu (Commented) (JIRA)

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

Zhihong Yu commented on HBASE-5077:
---

Patch looks good.

 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.1

 Attachments: HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread stack (Commented) (JIRA)

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

stack commented on HBASE-5077:
--

Chatting w/ J-D, we shouldn't return out of middle of finally -- should go 
through to end via the file closes.

 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.1

 Attachments: HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Hadoop QA (Commented) (JIRA)

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

Hadoop QA commented on HBASE-5077:
--

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12508163/HBASE-5077.patch
  against trunk revision .

+1 @author.  The patch does not contain any @author tags.

-1 tests included.  The patch doesn't appear to include any new or modified 
tests.
Please justify why no new tests are needed for this 
patch.
Also please list what manual steps were performed to 
verify this patch.

-1 javadoc.  The javadoc tool appears to have generated -152 warning 
messages.

+1 javac.  The applied patch does not increase the total number of javac 
compiler warnings.

-1 findbugs.  The patch appears to introduce 76 new Findbugs (version 
1.3.9) warnings.

+1 release audit.  The applied patch does not increase the total number of 
release audit warnings.

 -1 core tests.  The patch failed these unit tests:
   org.apache.hadoop.hbase.io.TestHeapSize

Test results: 
https://builds.apache.org/job/PreCommit-HBASE-Build/558//testReport/
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/558//artifact/trunk/patchprocess/newPatchFindbugsWarnings.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/558//console

This message is automatically generated.

 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.1

 Attachments: HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Hadoop QA (Commented) (JIRA)

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

Hadoop QA commented on HBASE-5077:
--

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12508167/HBASE-5077-v2.patch
  against trunk revision .

+1 @author.  The patch does not contain any @author tags.

-1 tests included.  The patch doesn't appear to include any new or modified 
tests.
Please justify why no new tests are needed for this 
patch.
Also please list what manual steps were performed to 
verify this patch.

-1 javadoc.  The javadoc tool appears to have generated -152 warning 
messages.

+1 javac.  The applied patch does not increase the total number of javac 
compiler warnings.

-1 findbugs.  The patch appears to introduce 76 new Findbugs (version 
1.3.9) warnings.

+1 release audit.  The applied patch does not increase the total number of 
release audit warnings.

 -1 core tests.  The patch failed these unit tests:
   org.apache.hadoop.hbase.io.TestHeapSize

Test results: 
https://builds.apache.org/job/PreCommit-HBASE-Build/560//testReport/
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/560//artifact/trunk/patchprocess/newPatchFindbugsWarnings.html
Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/560//console

This message is automatically generated.

 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.0

 Attachments: HBASE-5077-v2.patch, HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread stack (Commented) (JIRA)

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

stack commented on HBASE-5077:
--

+1 on patch for branch and trunk.

 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.0

 Attachments: HBASE-5077-v2.patch, HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Hudson (Commented) (JIRA)

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

Hudson commented on HBASE-5077:
---

Integrated in HBase-TRUNK #2565 (See 
[https://builds.apache.org/job/HBase-TRUNK/2565/])
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS -- fix 
compile error
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS

stack : 
Files : 
* 
/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* 
/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java


 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.0

 Attachments: HBASE-5077-v2.patch, HBASE-5077-v4.txt, HBASE-5077.patch


 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




[jira] [Commented] (HBASE-5077) SplitLogWorker fails to let go of a task, kills the RS

2011-12-20 Thread Hudson (Commented) (JIRA)

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

Hudson commented on HBASE-5077:
---

Integrated in HBase-0.92 #205 (See 
[https://builds.apache.org/job/HBase-0.92/205/])
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS -- fix 
compile error
HBASE-5077 SplitLogWorker fails to let go of a task, kills the RS

stack : 
Files : 
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java

stack : 
Files : 
* /hbase/branches/0.92/CHANGES.txt
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java


 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
Assignee: Jean-Daniel Cryans
Priority: Critical
 Fix For: 0.92.0

 Attachments: HBASE-5077-v2.patch, HBASE-5077-v4.txt, HBASE-5077.patch


 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