Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-14476

ReplicationQueuesZKImpl#copyQueuesFromRSUsingMulti will fail if there are orphaned queues under dead region server

VotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Closed
    • Minor
    • Resolution: Duplicate
    • 2.0.0
    • None
    • Replication
    • None

    Description

      ReplicationQueuesZKImpl#copyQueuesFromRSUsingMulti won't move the orphaned queues under dead region server(HBASE-12769 describes situations orphaned queues tend to happen):

              if (!peerExists(replicationQueueInfo.getPeerId())) {
                LOG.warn("Peer " + peerId + " didn't exist, skipping the replay");
                // Protection against moving orphaned queues
                continue;
              }
      

      After processing all the queues, the rsNode of dead region server will also be deleted:

            // 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);
      

      If there are orphaned queues, the rsNode of dead region server is not empty, so that the whole multi zookeeper operation will fail:

      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)
      

      This fail will make the normal queues under the dead region server can not be transferred if any orphaned queue exist.
      In 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:

                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;
                }
      

      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.

      Attachments

        1. HBASE-14476-trunk-v1.patch
          1 kB
          Jianwei Cui

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            Unassigned Unassigned
            cuijianwei Jianwei Cui
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment