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