Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5921918179 for ; Tue, 27 Oct 2015 08:24:31 +0000 (UTC) Received: (qmail 63856 invoked by uid 500); 27 Oct 2015 08:24:28 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 63785 invoked by uid 500); 27 Oct 2015 08:24:28 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 63761 invoked by uid 99); 27 Oct 2015 08:24:28 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 27 Oct 2015 08:24:28 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 042062C1F58 for ; Tue, 27 Oct 2015 08:24:28 +0000 (UTC) Date: Tue, 27 Oct 2015 08:24:28 +0000 (UTC) From: "Jianwei Cui (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (HBASE-14476) ReplicationQueuesZKImpl#copyQueuesFromRSUsingMulti will fail if there are orphaned queues under dead region server MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-14476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jianwei Cui updated HBASE-14476: -------------------------------- Attachment: HBASE-14476-trunk-v1.patch > ReplicationQueuesZKImpl#copyQueuesFromRSUsingMulti will fail if there are orphaned queues under dead region server > ------------------------------------------------------------------------------------------------------------------ > > Key: HBASE-14476 > URL: https://issues.apache.org/jira/browse/HBASE-14476 > Project: HBase > Issue Type: Improvement > Components: Replication > Affects Versions: 2.0.0 > Reporter: Jianwei Cui > Priority: Minor > Attachments: HBASE-14476-trunk-v1.patch > > > ReplicationQueuesZKImpl#copyQueuesFromRSUsingMulti won't move the orphaned queues under dead region server([HBASE-12769|https://issues.apache.org/jira/browse/HBASE-12769] describes situations orphaned queues tend to happen): > {code} > if (!peerExists(replicationQueueInfo.getPeerId())) { > LOG.warn("Peer " + peerId + " didn't exist, skipping the replay"); > // Protection against moving orphaned queues > continue; > } > {code} > After processing all the queues, the rsNode of dead region server will also be deleted: > {code} > // add delete op for dead rs, this will update the cversion of the parent. > // The reader will make optimistic locking with this to get a consistent > // snapshot > listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath)); > ... > ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); > {code} > If there are orphaned queues, the rsNode of dead region server is not empty, so that the whole multi zookeeper operation will fail: > {code} > 2015-09-23 20:17:55,170 WARN [ReplicationExecutor-0] replication.ReplicationQueuesZKImpl: Got exception in copyQueuesFromRSUsingMulti: > org.apache.zookeeper.KeeperException$NotEmptyException: KeeperErrorCode = Directory not empty > at org.apache.zookeeper.KeeperException.create(KeeperException.java:125) > at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:949) > at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:915) > {code} > This fail will make the normal queues under the dead region server can not be transferred if any orphaned queue exist. > In [HBASE-12865|https://issues.apache.org/jira/browse/HBASE-12865], ReplicationLogCleaner will depend the cversion change of rsNode parent to clean the WALs. Therefore, a possible solution is also transferring orphaned queues from dead region server. These orphaned queues will be skipped in ReplicationSourceManager$NodeFailoverWorker#run: > {code} > try { > peerConfig = replicationPeers.getReplicationPeerConfig(actualPeerId); > } catch (ReplicationException ex) { > LOG.warn("Received exception while getting replication peer config, skipping replay" > + ex); > } > if (peer == null || peerConfig == null) { > LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode); > continue; > } > {code} > This will make the orphaned queues also be kept in zookeeper with the queue name containing the transfer histories(waiting for manual operation), and the normal queues under the dead region server can also be processed. Suggestion and discussion are welcomed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)