hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jimmy Xiang (Commented) (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-5099) 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
Date Wed, 28 Dec 2011 00:53:30 GMT

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

Jimmy Xiang commented on HBASE-5099:
------------------------------------

After we restarted the master, distributed log splitting can go through and the system is
back to normal.

This can happen if zk session is expired, during tryRecoveringExpiredZKSession(), if it cannot
connect to the region server 
where root or meta is on, this region will be marked dead.  Then distributed log splitting
is kicked in.  But it won't go
through since the eventThread is waiting for the root/meta region assigned.

So this problem will happen whenever master tryRecoveringExpiredZKSession and the region server
with root/meta is not available.

One fix is to using sync call in accessing zk in distributed log splitting.  Another one is
to abort the master anyway
(don't try to recover since it may not be recoverable) if the region server with root/meta
region is not reachable.
                
> 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
>         Attachments: ZK-event-thread-waiting-for-root.png, distributed-log-splitting-hangs.png
>
>
> 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

        

Mime
View raw message