hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "jiraposter@reviews.apache.org (Commented) (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-5081) Distributed log splitting deleteNode races againsth splitLog retry
Date Thu, 22 Dec 2011 02:47:32 GMT

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

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

        

Mime
View raw message