[jira] [Commented] (HBASE-5081) Distributed log splitting deleteNode races againsth splitLog retry

2012-01-03 Thread Hadoop QA (Commented) (JIRA)

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

Hadoop QA commented on HBASE-5081:
--

-1 overall.  Here are the results of testing the latest attachment 
  
http://issues.apache.org/jira/secure/attachment/12509344/0001-HBASE-5081-jira-Distributed-log-splitting-deleteNode.patch
  against trunk revision .

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

+1 tests included.  The patch appears to include 8 new or modified tests.

-1 patch.  The patch command could not apply the patch.

Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/656//console

This message is automatically generated.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Prakash Khemani
 Fix For: 0.92.0

 Attachments: 
 0001-HBASE-5081-jira-Distributed-log-splitting-deleteNode.patch, 
 distributed-log-splitting-screenshot.png, hbase-5081-patch-v6.txt, 
 hbase-5081-patch-v7.txt, hbase-5081_patch_for_92_v4.txt, 
 hbase-5081_patch_v5.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hudson commented on HBASE-5081:
---

Integrated in HBase-0.92 #209 (See 
[https://builds.apache.org/job/HBase-0.92/209/])
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog 
retry; REVERT -- COMMITTED BEFORE REVIEW FINISHED -- AGAIN
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog retry; 
REAPPLY

stack : 
Files : 
* /hbase/branches/0.92/CHANGES.txt
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

stack : 
Files : 
* /hbase/branches/0.92/CHANGES.txt
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java


 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hudson commented on HBASE-5081:
---

Integrated in HBase-TRUNK #2568 (See 
[https://builds.apache.org/job/HBase-TRUNK/2568/])
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog 
retry; REVERT -- COMMITTED BEFORE REVIEW FINISHED -- AGAINpatch -p0 -R  x.txt 
patch -p0 -R  x.txt
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog retry; 
REAPPLY

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java


 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4076
---



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9178

Since the value of status won't change, I think it is better to call 
deleteNodeNow() here.
If we call deleteNodeNow() at line 360, we hold the lock much longer.


- Ted


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4077
---



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9179

DeleteAsyncCallback is only used by deleteNode().
I think we should simplify logic by removing deleteNode() and 
DeleteAsyncCallback - deleteNodeNow() uses RecoverableZooKeeper which has the 
retry logic.


- Ted


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-22 15:34:13, Ted Yu wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
373
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line373
bq.  
bq.   DeleteAsyncCallback is only used by deleteNode().
bq.   I think we should simplify logic by removing deleteNode() and 
DeleteAsyncCallback - deleteNodeNow() uses RecoverableZooKeeper which has the 
retry logic.

The difference is that deleteNode has unlimited retries.  RecoverableZooKeeper 
doesn't.  It has only 3 retries by default.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4077
---


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-22 15:08:38, Michael Stack wrote:
bq.   This patch seems to be raising too many questions.  Should we try going 
other route of ensuring the async delete removes the 'right' task?

Due to the race issue, we have to put more than the filename in the node and 
the hashmap, so as to removes the right task.
That's much bigger change and will raise more questions.


bq.  On 2011-12-22 15:08:38, Michael Stack wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
349
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line349
bq.  
bq.   Why check for success up here rather than down inside the 
synchronize on task.batch?   Why not do this safeToDeleteNodeAsync in there in 
the else clause where we up the count of errors?  Is it not safe to do the 
delete of zk node NOW under the synchronize block?

It is safe to do the delete under the synchronize block. The reason is that I 
don't want to hold the lock on task.batch while delete the node synchronously.  


bq.  On 2011-12-22 15:08:38, Michael Stack wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
378
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line378
bq.  
bq.   Are we duplicating the code from deleteNode here?  Should we have 
sync/async versions?

deleteNode is the async version.  deleteNodeNow is the sync version.  The async 
version can have unlimited retries.  The sync version can retry up to certain 
configured number (3 by default).
So the sync version doesn't guarantee it will be deleted.  The code wise, it's 
hard to reuse.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4073
---


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs 

[jira] [Commented] (HBASE-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

stack commented on HBASE-5081:
--

@Jimmy Another tack would be ensuring splitLogDistributed has cleaned-up after 
itself before it returns including clean up on early-out because of exception.  
It seems like we will rerun the split if we early-out if 
OrphanHLogAfterSplitException is thrown ONLY (Is this what happened in your 
scenario?  You say three log splits failed?  Was it because a new log file 
showed up: i.e. OrphanHLogAfterSplitException?  Or for some other reason?  If 
for some other reason, the split should have failed?).  I'd think that if a new 
file shows up while we were splitting, its fine to redo the split but I'd think 
that splitLogDistibuted would make sure it'd cleaned up after itself before it 
returned... that it had completed the batch it had been asked do.

I was waiting on this issue to be done before cutting the RC but after looking 
at the pieces, I think that while this an important issue, my thinking is that 
it rare so I won't hold up the RC.

Good stuff.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4089
---


I feel that the proper fix should go in the method createTaskIfAbsent()

before attempting to delete a task in zk, task.deleted is set to true. The task 
is not removed from tasks array until task is successfully removed from zk.

In createTaskIfAbsent() when you find a deleted task we should do the following
* If the task had completed successfully then return null. (It is as if the 
task is completed right away).
* if the task had completed unsuccessfully then block (with timeouts) until the 
task is removed from the tasks array.

Without fixing anything, the problem, I think is present only in the following 
scenario
- at startup the master acquires orphan tasks  listed in zookeeper. One of 
these orphan tasks fails. Before that orphan task could be deleted some master 
thread asks for that task to be completed. As  things currently stand, the 
SplitLogManager will reply with SUCCESS immediately. (This is because of the 
logic in createTaskIfAbsent())

The common case where  this race happens should work ...
- a master thread asks for a log dir to be split. That task fails but it has 
not been deleted from zk yet nor removed from tasks yet. The log-dir-split is 
retried and the retry finds the old, soon to be deleted task. But the retry 
will also see that task.batch is set and it will immediately throw an error 
saying 'someone else is waiting for this task'. And the next time log-dir-split 
is retried the tasks map might have been cleared and things will work.


src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9193

The task corresponding to this path has to be removed from the tasks map 
(as in deleteNodeSuccess())



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9194

I guess this should be considered an error that the delete did not go 
through?


- Prakash


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly 

[jira] [Commented] (HBASE-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread Lars Hofhansl (Commented) (JIRA)

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

Lars Hofhansl commented on HBASE-5081:
--

@Prakash: I like that idea. Keeps the logic where it belongs.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-22 22:55:55, Prakash Khemani wrote:
bq.   I feel that the proper fix should go in the method createTaskIfAbsent()
bq.   
bq.   before attempting to delete a task in zk, task.deleted is set to true. 
The task is not removed from tasks array until task is successfully removed 
from zk.
bq.   
bq.   In createTaskIfAbsent() when you find a deleted task we should do the 
following
bq.   * If the task had completed successfully then return null. (It is as if 
the task is completed right away).
bq.   * if the task had completed unsuccessfully then block (with timeouts) 
until the task is removed from the tasks array.
bq.   
bq.   Without fixing anything, the problem, I think is present only in the 
following scenario
bq.   - at startup the master acquires orphan tasks  listed in zookeeper. One 
of these orphan tasks fails. Before that orphan task could be deleted some 
master thread asks for that task to be completed. As  things currently stand, 
the SplitLogManager will reply with SUCCESS immediately. (This is because of 
the logic in createTaskIfAbsent())
bq.   
bq.   The common case where  this race happens should work ...
bq.   - a master thread asks for a log dir to be split. That task fails but it 
has not been deleted from zk yet nor removed from tasks yet. The log-dir-split 
is retried and the retry finds the old, soon to be deleted task. But the retry 
will also see that task.batch is set and it will immediately throw an error 
saying 'someone else is waiting for this task'. And the next time log-dir-split 
is retried the tasks map might have been cleared and things will work.

The task is not removed from tasks array until task is successfully removed 
from zk.

This seems not correct.  stopTrackingTasks() will remove all tasks even if the 
task is not removed from zk.
That's why createTaskIfAbsent() can put a new task in the set.

If we remove stopTrackingTasks(), then the task should be still in tasks, then 
this alternative will work.
Will removing stopTrackingTasks() cause other issues?  For the second *, how 
long should we block?  If
the task is still not removed from the tasks array after the timeout, what 
should we do?

Can you come up a patch? I am very open to any fix.


bq.  On 2011-12-22 22:55:55, Prakash Khemani wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
382
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line382
bq.  
bq.   The task corresponding to this path has to be removed from the tasks 
map (as in deleteNodeSuccess())

It is removed in the stopTrackingTasks() methods, since this one is failed, so 
batch.installed != batch.done.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4089
---


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 

[jira] [Commented] (HBASE-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread Jimmy Xiang (Commented) (JIRA)

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

Jimmy Xiang commented on HBASE-5081:


@Stack, it is not an orphan task.  It happens in ServerShutdownHandler.  It 
retries the log splitting if the previous one failed for any reason:

line 178:
this.services.getExecutorService().submit(this);

It keep retrying.  Should we have a limit here?

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-22 Thread Jimmy Xiang (Commented) (JIRA)

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

Jimmy Xiang commented on HBASE-5081:


@Prakash, this one didn't happen when the master starts up.  It happened when 
one region server died.


 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Zhihong Yu commented on HBASE-5081:
---

Currently SplitLogManager.Task doesn't have flag indicating whether the 
deletion was caused by successful splitting.
I think we should introduce such flag through boolean or enum so that 
stopTrackingTasks() can make better decision.

Looking at the code from 0.89-fb, I can see some subtle differences between 
0.89-fb and TRUNK.
e.g. task.batch.notify() in 0.89-fb is guarded by a condition:
{code}
  if (!task.isOrphan()) {
synchronized (task.batch) {
  if (status == SUCCESS) {
task.batch.done++;
  } else {
task.batch.error++;
  }
  if ((task.batch.done + task.batch.error) == task.batch.installed) {
task.batch.notify();
  }
}
  }
{code}
I think we should unify the two codebases so that our observations have common 
ground.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

stack commented on HBASE-5081:
--

@Jimmy I see. Its a recursion on ioe;  Any ioe could get us stuck here retrying 
split logs of a downed server.  I suppose there should be a bound on how many 
times we try split logs or a particular server.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread Jimmy Xiang (Commented) (JIRA)

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

Jimmy Xiang commented on HBASE-5081:


I am working on a path now. I think synchronous deleteNode is clean.  It will 
give retry a fresh start.
But it may take a while if there are too many files.  Yes, for long term, we 
can think about how to do what stack says.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread Jimmy Xiang (Commented) (JIRA)

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

Jimmy Xiang commented on HBASE-5081:


Can we deleteNode only if it is successfully done?  If it is not completed, let 
the node stay there.  In this case, when the retry happens, it should see the 
old node there, but it is ok.  The new task in the hashmap won't be deleted 
either.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/
---

Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.


Summary
---

In this patch, after a task is done, we don't delete the node if the task is 
failed.  So that when it's retried later on, there won't be race problem.

It used to delete the node always.


This addresses bug HBASE-5081.
https://issues.apache.org/jira/browse/HBASE-5081


Diffs
-

  src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 7b7316f 

Diff: https://reviews.apache.org/r/3292/diff


Testing
---

mvn -Dtest=TestDistributedLogSplitting clean test


Thanks,

Jimmy



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 patch_for_92.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/
---

(Updated 2011-12-21 17:01:22.901024)


Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.


Changes
---

Updated the comments.


Summary
---

In this patch, after a task is done, we don't delete the node if the task is 
failed.  So that when it's retried later on, there won't be race problem.

It used to delete the node always.


This addresses bug HBASE-5081.
https://issues.apache.org/jira/browse/HBASE-5081


Diffs (updated)
-

  src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 7b7316f 

Diff: https://reviews.apache.org/r/3292/diff


Testing
---

mvn -Dtest=TestDistributedLogSplitting clean test


Thanks,

Jimmy



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 patch_for_92.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4046
---



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9151

White space on end and 'is' should be 'if'


- Michael


On 2011-12-21 17:01:22, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-21 17:01:22)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
7b7316f 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 patch_for_92.txt, patch_for_92_v2.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4045
---

Ship it!


Looks good to me.  How I know it works?  Would it be hard to write a test?

- Michael


On 2011-12-21 17:01:22, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-21 17:01:22)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
7b7316f 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 patch_for_92.txt, patch_for_92_v2.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-21 17:06:22, Michael Stack wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
358
bq.   https://reviews.apache.org/r/3292/diff/2/?file=65660#file65660line358
bq.  
bq.   White space on end and 'is' should be 'if'

Let me fix it.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4046
---


On 2011-12-21 17:01:22, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-21 17:01:22)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
7b7316f 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 patch_for_92.txt, patch_for_92_v2.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/
---

(Updated 2011-12-21 17:12:36.185307)


Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.


Changes
---

Fixed the comments.


Summary
---

In this patch, after a task is done, we don't delete the node if the task is 
failed.  So that when it's retried later on, there won't be race problem.

It used to delete the node always.


This addresses bug HBASE-5081.
https://issues.apache.org/jira/browse/HBASE-5081


Diffs (updated)
-

  src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 7b7316f 

Diff: https://reviews.apache.org/r/3292/diff


Testing
---

mvn -Dtest=TestDistributedLogSplitting clean test


Thanks,

Jimmy



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 patch_for_92.txt, patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4050
---


Interesting idea. Let us know result of testing in real cluster.
Since the zk node may live across cluster restart, we should verify the change 
works in that scenario.


src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9158

I think the new comments should be lifted to line 353 for clarity.



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9156

Should read 'if the task failed'



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9157

Indentation should be 2 spaces.


- Ted


On 2011-12-21 17:12:36, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-21 17:12:36)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
7b7316f 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 patch_for_92.txt, patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/
---

(Updated 2011-12-21 17:40:59.028814)


Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.


Changes
---

Fixed the indentation issue, added more comment.


Summary
---

In this patch, after a task is done, we don't delete the node if the task is 
failed.  So that when it's retried later on, there won't be race problem.

It used to delete the node always.


This addresses bug HBASE-5081.
https://issues.apache.org/jira/browse/HBASE-5081


Diffs (updated)
-

  src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 667a8b1 

Diff: https://reviews.apache.org/r/3292/diff


Testing
---

mvn -Dtest=TestDistributedLogSplitting clean test


Thanks,

Jimmy



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread Jimmy Xiang (Commented) (JIRA)

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

Jimmy Xiang commented on HBASE-5081:


The patch is for both 0.92 and 0.94 actually.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4052
---



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9159

Should read 'Asynchronously deleting'



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9160

Should read 'if the task failed and was not an orphan'


- Ted


On 2011-12-21 17:40:59, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-21 17:40:59)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hadoop QA commented on HBASE-5081:
--

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12508264/patch_for_92_v2.txt
  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.replication.TestReplication
  org.apache.hadoop.hbase.replication.TestMultiSlaveReplication
  org.apache.hadoop.hbase.replication.TestMasterReplication

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

This message is automatically generated.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/
---

(Updated 2011-12-21 18:06:55.460515)


Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.


Changes
---

Changed some comments


Summary
---

In this patch, after a task is done, we don't delete the node if the task is 
failed.  So that when it's retried later on, there won't be race problem.

It used to delete the node always.


This addresses bug HBASE-5081.
https://issues.apache.org/jira/browse/HBASE-5081


Diffs (updated)
-

  src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 667a8b1 

Diff: https://reviews.apache.org/r/3292/diff


Testing
---

mvn -Dtest=TestDistributedLogSplitting clean test


Thanks,

Jimmy



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-21 17:44:53, Ted Yu wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
368
bq.   https://reviews.apache.org/r/3292/diff/4/?file=65674#file65674line368
bq.  
bq.   Should read 'if the task failed and was not an orphan'

The original is fine.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4052
---


On 2011-12-21 18:06:55, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-21 18:06:55)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hadoop QA commented on HBASE-5081:
--

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12508259/patch_for_92.txt
  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.client.TestInstantSchemaChange
  org.apache.hadoop.hbase.mapred.TestTableMapReduce
  org.apache.hadoop.hbase.mapreduce.TestHFileOutputFormat
  org.apache.hadoop.hbase.master.TestSplitLogManager

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

This message is automatically generated.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Zhihong Yu commented on HBASE-5081:
---

I can reproduce the following test failures on MacBook:
{code}
Failed tests:   testTaskErr(org.apache.hadoop.hbase.master.TestSplitLogManager)
  testUnassignedTimeout(org.apache.hadoop.hbase.master.TestSplitLogManager)
{code}
Please adjust TestSplitLogManager accordingly.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread Jimmy Xiang (Commented) (JIRA)

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

Jimmy Xiang commented on HBASE-5081:


I am thinking about sync delete for the failure case.  What do you think?

I am adjusting the test failure now.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hadoop QA commented on HBASE-5081:
--

-1 overall.  Here are the results of testing the latest attachment 
  
http://issues.apache.org/jira/secure/attachment/12508297/hbase-5081-patch-v6.txt
  against trunk revision .

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

+1 tests included.  The patch appears to include 3 new or modified tests.

-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.replication.TestReplication

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

This message is automatically generated.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081_patch_for_92_v4.txt, 
 hbase-5081_patch_v5.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread Jimmy Xiang (Commented) (JIRA)

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

Jimmy Xiang commented on HBASE-5081:


Upload patch v6 to review board.

+assertTrue(ZKUtil.checkExists(zkw, tasknode) != -1);

The original is this:

+assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);

This assertion can be satisfied when there is no race without this patch.
With this patch, we don't delete any failed task node.  So the node should be
there now.

I am still working on a patch to delete the node synchronously in this scenario.




 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081_patch_for_92_v4.txt, 
 hbase-5081_patch_v5.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4059
---


Test change looks good to me.

- Lars


On 2011-12-21 21:30:56, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-21 21:30:56)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
7b7316f 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
5c9d7dd 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081_patch_for_92_v4.txt, 
 hbase-5081_patch_v5.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hadoop QA commented on HBASE-5081:
--

-1 overall.  Here are the results of testing the latest attachment 
  
http://issues.apache.org/jira/secure/attachment/12508273/hbase-5081_patch_v5.txt
  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.coprocessor.TestCoprocessorEndpoint
  org.apache.hadoop.hbase.mapred.TestTableMapReduce
  org.apache.hadoop.hbase.mapreduce.TestHFileOutputFormat
  org.apache.hadoop.hbase.master.TestSplitLogManager

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

This message is automatically generated.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081_patch_for_92_v4.txt, 
 hbase-5081_patch_v5.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Zhihong Yu commented on HBASE-5081:
---

Do we have a test case where zk nodes (including the failed task node) survive 
cluster restart and distributed log splitting successfully finishes after 
cluster restart ?

If we use patch v6 instead of the synchronous node deletion patch (upcoming), 
we should check the value of resubmit_threshold. If the value is 0, we should 
still delete the failed task node because there would be no resubmission.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081_patch_for_92_v4.txt, 
 hbase-5081_patch_v5.txt, patch_for_92.txt, patch_for_92_v2.txt, 
 patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-21 17:05:47, Michael Stack wrote:
bq.   Looks good to me.  How I know it works?  Would it be hard to write a 
test?
bq.  
bq.  Jimmy Xiang wrote:
bq.  This is a race issue hard to reproduce. Let me think about how to come 
up a unit test.

Jimmy.  Please paste your last patch to JIRA.  I'd like to include in our 
0.92RC though it doesn't have a test.


- Michael


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4045
---


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-21 17:05:47, Michael Stack wrote:
bq.   Looks good to me.  How I know it works?  Would it be hard to write a 
test?
bq.  
bq.  Jimmy Xiang wrote:
bq.  This is a race issue hard to reproduce. Let me think about how to come 
up a unit test.
bq.  
bq.  Michael Stack wrote:
bq.  Jimmy.  Please paste your last patch to JIRA.  I'd like to include in 
our 0.92RC though it doesn't have a test.

Yes, it is in Jira: hbase-5081-patch-v7.txt
Thanks.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4045
---


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hadoop QA commented on HBASE-5081:
--

-1 overall.  Here are the results of testing the latest attachment 
  
http://issues.apache.org/jira/secure/attachment/12508328/hbase-5081-patch-v7.txt
  against trunk revision .

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

+1 tests included.  The patch appears to include 3 new or modified tests.

-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.replication.TestReplication

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

This message is automatically generated.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4064
---



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9163

Did the first approach not work?



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9164

It seems we can do this unconditionally now (no need for the 
safeToDeleteNodeAsync flag). The worst scenario is trying to remove a node that 
has already been removed


- Lars


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-22 00:58:56, Lars Hofhansl wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
373
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line373
bq.  
bq.   It seems we can do this unconditionally now (no need for the 
safeToDeleteNodeAsync flag). The worst scenario is trying to remove a node that 
has already been removed

If the failed node is not removed at the beginning, we could run into the same 
race issue again.


bq.  On 2011-12-22 00:58:56, Lars Hofhansl wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
354
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line354
bq.  
bq.   Did the first approach not work?

The first approach will not hang the splitLog method, but the failed tasks 
won't be actually tried again since the state stays in TaskState.TASK_ERR.  We 
do need to delete those nodes unless we put different data in zookeeper as 
Stack suggested.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4064
---


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 

[jira] [Commented] (HBASE-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4067
---



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9167

The deletion is immediate. Should this counter be incremented ?



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9168

I think we should be more cautious because RecoverableZooKeeper has 
attempted retry.
ke should be rethrown.


- Ted


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--



bq.  On 2011-12-22 02:23:19, Ted Yu wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
380
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line380
bq.  
bq.   The deletion is immediate. Should this counter be incremented ?

Yes, so that we can track how many deletions succeed, how many fail.


bq.  On 2011-12-22 02:23:19, Ted Yu wrote:
bq.   src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java, line 
386
bq.   https://reviews.apache.org/r/3292/diff/8/?file=65682#file65682line386
bq.  
bq.   I think we should be more cautious because RecoverableZooKeeper has 
attempted retry.
bq.   ke should be rethrown.

In this case, we should not re-throw it actually.  In the corresponding 
asynchronous deleteNode method, it doesn't throw KeeperException either. It 
just logs the failure.


- Jimmy


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4067
---


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

2011-12-21 Thread jirapos...@reviews.apache.org (Commented) (JIRA)

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

jirapos...@reviews.apache.org commented on HBASE-5081:
--


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3292/#review4069
---



src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
https://reviews.apache.org/r/3292/#comment9171

Looking at current code:
  private void deleteNodeFailure(String path) {
LOG.fatal(logic failure, failing to delete a node should never happen 
 +
because delete has infinite retries);

since the above failure should never happen, the correct action would be to 
rethrown the exception - for both synchronous and async cases.


- Ted


On 2011-12-22 00:31:23, Jimmy Xiang wrote:
bq.  
bq.  ---
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/3292/
bq.  ---
bq.  
bq.  (Updated 2011-12-22 00:31:23)
bq.  
bq.  
bq.  Review request for hbase, Ted Yu, Michael Stack, and Lars Hofhansl.
bq.  
bq.  
bq.  Summary
bq.  ---
bq.  
bq.  In this patch, after a task is done, we don't delete the node if the task 
is failed.  So that when it's retried later on, there won't be race problem.
bq.  
bq.  It used to delete the node always.
bq.  
bq.  
bq.  This addresses bug HBASE-5081.
bq.  https://issues.apache.org/jira/browse/HBASE-5081
bq.  
bq.  
bq.  Diffs
bq.  -
bq.  
bq.src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java 
667a8b1 
bq.src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java 
32ad7e8 
bq.  
bq.  Diff: https://reviews.apache.org/r/3292/diff
bq.  
bq.  
bq.  Testing
bq.  ---
bq.  
bq.  mvn -Dtest=TestDistributedLogSplitting clean test
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Jimmy
bq.  
bq.



 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hudson commented on HBASE-5081:
---

Integrated in HBase-TRUNK #2567 (See 
[https://builds.apache.org/job/HBase-TRUNK/2567/])
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog 
retry; REVERT -- COMMITTED BEFORE REVIEW FINISHED
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog retry

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java


 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

stack commented on HBASE-5081:
--

Ugh.  Sorry to whoever is trying to follow along.  I applied the patch a few 
hours ago and then backed it out because it looked like ongoing review still 
(Lars had comments).  I then just reapplied it because Lars gave +1 but looks 
like Ted and Jimmy are still chatting above so I backed it out again for now.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hudson commented on HBASE-5081:
---

Integrated in HBase-0.92 #208 (See 
[https://builds.apache.org/job/HBase-0.92/208/])
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog 
retry; REVERT -- COMMITTED BEFORE REVIEW FINISHED
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog retry

stack : 
Files : 
* /hbase/branches/0.92/CHANGES.txt
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

stack : 
Files : 
* /hbase/branches/0.92/CHANGES.txt
* 
/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java


 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Zhihong Yu commented on HBASE-5081:
---

I can understand the following code from HLogSplitter.java:
{code}
LOG.fatal(this.getName() +  Got while writing log entry to log, e);
throw e;
{code}
But not this code from SplitLogWorker.java:
{code}
} catch (KeeperException.NoNodeException e) {
  LOG.fatal(logic error - end task  + path +   + ts +
   failed because task doesn't exist, e);
} catch (KeeperException e) {
{code}
I think we should add rethrowing in the above case for NoNodeException in this 
JIRA.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Zhihong Yu commented on HBASE-5081:
---

I found 4 other occasions of LOG.fatal() without (re)throwing in 
SplitLogManager.java, such as:
{code}
public void setBatch(TaskBatch batch) {
  if (batch != null  this.batch != null) {
LOG.fatal(logic error - batch being overwritten);
  }
{code}
I think these 4 places should throw proper exception.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

stack commented on HBASE-5081:
--

Above comments seem a little outside the scope of this issue Zhihong?

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Hudson commented on HBASE-5081:
---

Integrated in HBase-TRUNK-security #40 (See 
[https://builds.apache.org/job/HBase-TRUNK-security/40/])
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog 
retry; REVERT -- COMMITTED BEFORE REVIEW FINISHED -- AGAINpatch -p0 -R  x.txt 
patch -p0 -R  x.txt
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog retry; 
REAPPLY
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog retry; 
REVERT -- COMMITTED BEFORE REVIEW FINISHED
HBASE-5081 Distributed log splitting deleteNode races againsth splitLog retry

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
* 
/hbase/trunk/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java


 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Fix For: 0.92.0

 Attachments: distributed-log-splitting-screenshot.png, 
 hbase-5081-patch-v6.txt, hbase-5081-patch-v7.txt, 
 hbase-5081_patch_for_92_v4.txt, hbase-5081_patch_v5.txt, patch_for_92.txt, 
 patch_for_92_v2.txt, patch_for_92_v3.txt


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

stack commented on HBASE-5081:
--

Or we need a means of uniquely identifying the failed split in the hashmap so 
when the callback runs, it only removes the pertinent tasks if present; i.e. 
the filename alone is not enough?

Otherwise, sounds good Jimmy.  Good find.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

--
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-5081) Distributed log splitting deleteNode races againsth splitLog retry

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

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

Zhihong Yu commented on HBASE-5081:
---

To expedite the release of 0.92, I think we can make deleteNode synchronous 
first.

Once distributed log splitting works robustly, we can implement async 
deleteNode in a follow on JIRA.

 Distributed log splitting deleteNode races againsth splitLog retry 
 ---

 Key: HBASE-5081
 URL: https://issues.apache.org/jira/browse/HBASE-5081
 Project: HBase
  Issue Type: Bug
  Components: wal
Affects Versions: 0.92.0, 0.94.0
Reporter: Jimmy Xiang
Assignee: Jimmy Xiang
 Attachments: distributed-log-splitting-screenshot.png


 Recently, during 0.92 rc testing, we found distributed log splitting hangs 
 there forever.  Please see attached screen shot.
 I looked into it and here is what happened I think:
 1. One rs died, the servershutdownhandler found it out and started the 
 distributed log splitting;
 2. All three tasks failed, so the three tasks were deleted, asynchronously;
 3. Servershutdownhandler retried the log splitting;
 4. During the retrial, it created these three tasks again, and put them in a 
 hashmap (tasks);
 5. The asynchronously deletion in step 2 finally happened for one task, in 
 the callback, it removed one
 task in the hashmap;
 6. One of the newly submitted tasks' zookeeper watcher found out that task is 
 unassigned, and it is not
 in the hashmap, so it created a new orphan task.
 7.  All three tasks failed, but that task created in step 6 is an orphan so 
 the batch.err counter was one short,
 so the log splitting hangs there and keeps waiting for the last task to 
 finish which is never going to happen.
 So I think the problem is step 2.  The fix is to make deletion sync, instead 
 of async, so that the retry will have
 a clean start.
 Async deleteNode will mess up with split log retrial.  In extreme situation, 
 if async deleteNode doesn't happen
 soon enough, some node created during the retrial could be deleted.
 deleteNode should be sync.

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