[
https://issues.apache.org/jira/browse/HBASE-5099?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13177431#comment-13177431
]
[email protected] commented on HBASE-5099:
------------------------------------------------------
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/3323/#review4154
-----------------------------------------------------------
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<https://reviews.apache.org/r/3323/#comment9383>
Should we issue executor.shutdownNow() here if the worker did not finish,
in order to attempt to interrupt and long running worker?
- Lars
On 2011-12-29 21:16:22, Jimmy Xiang wrote:
bq.
bq. -----------------------------------------------------------
bq. This is an automatically generated e-mail. To reply, visit:
bq. https://reviews.apache.org/r/3323/
bq. -----------------------------------------------------------
bq.
bq. (Updated 2011-12-29 21:16:22)
bq.
bq.
bq. Review request for hbase, Ted Yu and Michael Stack.
bq.
bq.
bq. Summary
bq. -------
bq.
bq. Per discussion with Ted (on issues), I put up a patch to run
tryRecoveringExpiredZKSession() in a separate thread and time it out and fail
the recovery if it is stuck somewhere.
bq.
bq. I added a test to test the abort method. However, for the mini cluster,
becomeActiveMaster() doesn't succeed so the abort method ends up always
aborted. So the actually success recovery is not tested.
bq.
bq.
bq. This addresses bug HBASE-5099.
bq. https://issues.apache.org/jira/browse/HBASE-5099
bq.
bq.
bq. Diffs
bq. -----
bq.
bq. src/main/java/org/apache/hadoop/hbase/master/HMaster.java a5935a6
bq.
src/test/java/org/apache/hadoop/hbase/master/TestMasterZKSessionRecovery.java
PRE-CREATION
bq.
bq. Diff: https://reviews.apache.org/r/3323/diff
bq.
bq.
bq. Testing
bq. -------
bq.
bq. mvn -PlocalTests -Dtest=TestMaster* clean test
bq.
bq.
bq. Thanks,
bq.
bq. Jimmy
bq.
bq.
> ZK event thread waiting for root region while server shutdown handler waiting
> for event thread to finish distributed log splitting to recover the region
> sever the root region is on
> ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
>
> Key: HBASE-5099
> URL: https://issues.apache.org/jira/browse/HBASE-5099
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.92.0, 0.94.0
> Reporter: Jimmy Xiang
> Assignee: Jimmy Xiang
> Fix For: 0.92.0, 0.94.0
>
> Attachments: 5099.92, ZK-event-thread-waiting-for-root.png,
> distributed-log-splitting-hangs.png, hbase-5099-v2.patch,
> hbase-5099-v3.patch, hbase-5099-v4.patch, hbase-5099-v5.patch,
> hbase-5099.patch
>
>
> A RS died. The ServerShutdownHandler kicked in and started the logspliting.
> SpliLogManager
> installed the tasks asynchronously, then started to wait for them to complete.
> The task znodes were not created actually. The requests were just queued.
> At this time, the zookeeper connection expired. HMaster tried to recover the
> expired ZK session.
> During the recovery, a new zookeeper connection was created. However, this
> master became the
> new master again. It tried to assign root and meta.
> Because the dead RS got the old root region, the master needs to wait for the
> log splitting to complete.
> This waiting holds the zookeeper event thread. So the async create split
> task is never retried since
> there is only one event thread, which is waiting for the root region assigned.
--
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