hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ted Yu (JIRA)" <j...@apache.org>
Subject [jira] [Reopened] (HBASE-12241) The crash of regionServer when taking deadserver's replication queue breaks replication
Date Thu, 16 Oct 2014 21:40:34 GMT

     [ https://issues.apache.org/jira/browse/HBASE-12241?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Ted Yu reopened HBASE-12241:
----------------------------

[~lshmouse]:
Can you attach a doc patch ?

> The crash of regionServer when taking deadserver's replication queue breaks replication
> ---------------------------------------------------------------------------------------
>
>                 Key: HBASE-12241
>                 URL: https://issues.apache.org/jira/browse/HBASE-12241
>             Project: HBase
>          Issue Type: Bug
>          Components: Replication
>            Reporter: Liu Shaohui
>            Assignee: Liu Shaohui
>            Priority: Critical
>             Fix For: 2.0.0, 0.99.2
>
>         Attachments: HBASE-12241-trunk-v1.diff
>
>
> When a regionserver crash, another regionserver will try to take over the replication
hlogs queue and help the the the dead regionserver to finish the replcation.See NodeFailoverWorker
in ReplicationSourceManager
> Currently hbase.zookeeper.useMulti is false in default configuration. The operation of
taking over replication queue is not atomic. The ReplicationSourceManager firstly lock the
replication node of dead regionserver and then copy the replication queue, and delete replication
node of dead regionserver at last. The operation of the lockOtherRS just creates a persistent
zk node named "lock" which prevent other regionserver taking over the replication queue.
> See:
> {code}
>   public boolean lockOtherRS(String znode) {
>     try {
>       String parent = ZKUtil.joinZNode(this.rsZNode, znode);
>       if (parent.equals(rsServerNameZnode)) {
>         LOG.warn("Won't lock because this is us, we're dead!");
>         return false;
>       }
>       String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
>       ZKUtil.createAndWatch(this.zookeeper, p, Bytes.toBytes(rsServerNameZnode));
>     } catch (KeeperException e) {
>       ...
>       return false;
>     }
>     return true;
>   }
> {code}
> But if a regionserver crashed after creating this "lock" zk node and before coping the
replication queue to its replication queue, the "lock" zk node will be left forever and
> no other regionserver can take over the replication queue.
> In out production cluster, we encounter this problem. We found the replication queue
was there and no regionserver took over it and a "lock" zk node left there.
> {quote}
> hbase.32561.log:2014-09-24,14:09:28,790 INFO org.apache.hadoop.hbase.replication.ReplicationZookeeper:
Won't transfer the queue, another RS took care of it because of: KeeperErrorCode = NoNode
for /hbase/hhsrv-micloud/replication/rs/hh-hadoop-srv-st09.bj,12610,1410937824255/lock
> hbase.32561.log:2014-09-24,14:14:45,148 INFO org.apache.hadoop.hbase.replication.ReplicationZookeeper:
Won't transfer the queue, another RS took care of it because of: KeeperErrorCode = NoNode
for /hbase/hhsrv-micloud/replication/rs/hh-hadoop-srv-st10.bj,12600,1410937795685/lock
> {quote}
> A quick solution is that the lock operation just create an ephemeral "lock" zookeeper
node and when the lock node is deleted, other regionserver will be notified to check if there
are replication queue left.
> Suggestions are welcomed! Thanks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message