hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jianwei Cui (JIRA)" <j...@apache.org>
Subject [jira] [Created] (HBASE-14476) ReplicationQueuesZKImpl#copyQueuesFromRSUsingMulti will fail if there are orphaned queues under dead region server
Date Thu, 24 Sep 2015 02:28:04 GMT
Jianwei Cui created HBASE-14476:
-----------------------------------

             Summary: 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


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)

Mime
View raw message