View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.master.cleaner;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.HashMap;
23  import java.util.HashSet;
24  import java.util.List;
25  import java.util.Map;
26  import java.util.Set;
27  import java.util.Map.Entry;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.Abortable;
33  import org.apache.hadoop.hbase.HConstants;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.replication.ReplicationFactory;
36  import org.apache.hadoop.hbase.replication.ReplicationPeers;
37  import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
38  import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
39  import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
40  import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
41  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
42  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
43  import org.apache.zookeeper.KeeperException;
44  
45  /**
46   * Used to clean the replication queues belonging to the peer which does not exist.
47   */
48  @InterfaceAudience.Private
49  public class ReplicationZKNodeCleaner {
50    private static final Log LOG = LogFactory.getLog(ReplicationZKNodeCleaner.class);
51    private final ZooKeeperWatcher zkw;
52    private final ReplicationQueuesClient queuesClient;
53    private final ReplicationPeers replicationPeers;
54    private final ReplicationQueueDeletor queueDeletor;
55    private final boolean useMulti;
56  
57    public ReplicationZKNodeCleaner(Configuration conf, ZooKeeperWatcher zkw, Abortable abortable)
58        throws IOException {
59      try {
60        this.zkw = zkw;
61        this.queuesClient = ReplicationFactory
62            .getReplicationQueuesClient(zkw, conf, abortable);
63        this.queuesClient.init();
64        this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.queuesClient,
65          abortable);
66        this.replicationPeers.init();
67        this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable);
68        this.useMulti = conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
69      } catch (Exception e) {
70        throw new IOException("failed to construct ReplicationZKNodeCleaner", e);
71      }
72    }
73  
74    /**
75     * @return undeletedQueues replicator with its queueIds for removed peers
76     * @throws IOException
77     */
78    public Map<String, List<String>> getUnDeletedQueues() throws IOException {
79      Map<String, List<String>> undeletedQueues = new HashMap<>();
80      Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
81      try {
82        List<String> replicators = this.queuesClient.getListOfReplicators();
83        if (replicators == null || replicators.isEmpty()) {
84          return undeletedQueues;
85        }
86        for (String replicator : replicators) {
87          List<String> queueIds = this.queuesClient.getAllQueues(replicator);
88          for (String queueId : queueIds) {
89            if (!useMulti && queueId.equals(ReplicationQueuesZKImpl.RS_LOCK_ZNODE)) {
90              continue;
91            }
92            ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
93            if (!peerIds.contains(queueInfo.getPeerId())) {
94              if (!undeletedQueues.containsKey(replicator)) {
95                undeletedQueues.put(replicator, new ArrayList<String>());
96              }
97              undeletedQueues.get(replicator).add(queueId);
98              if (LOG.isDebugEnabled()) {
99                LOG.debug("Undeleted replication queue for removed peer found: "
100                   + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]",
101                     queueInfo.getPeerId(), replicator, queueId));
102             }
103           }
104         }
105       }
106     } catch (KeeperException ke) {
107       throw new IOException("Failed to get the replication queues of all replicators", ke);
108     }
109     return undeletedQueues;
110   }
111 
112   /**
113    * @return undeletedHFileRefsQueue replicator with its undeleted queueIds for removed peers in
114    *         hfile-refs queue
115    * @throws IOException
116    */
117   public Set<String> getUnDeletedHFileRefsQueues() throws IOException {
118     Set<String> undeletedHFileRefsQueue = new HashSet<>();
119     Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
120     String hfileRefsZNode = queueDeletor.getHfileRefsZNode();
121     try {
122       if (-1 == ZKUtil.checkExists(zkw, hfileRefsZNode)) {
123         return null;
124       }
125       List<String> listOfPeers = this.queuesClient.getAllPeersFromHFileRefsQueue();
126       Set<String> peers = new HashSet<>(listOfPeers);
127       peers.removeAll(peerIds);
128       if (!peers.isEmpty()) {
129         undeletedHFileRefsQueue.addAll(peers);
130       }
131     } catch (KeeperException e) {
132       throw new IOException("Failed to get list of all peers from hfile-refs znode "
133           + hfileRefsZNode, e);
134     }
135     return undeletedHFileRefsQueue;
136   }
137 
138   private class ReplicationQueueDeletor extends ReplicationStateZKBase {
139 
140     public ReplicationQueueDeletor(ZooKeeperWatcher zk, Configuration conf, Abortable abortable) {
141       super(zk, conf, abortable);
142     }
143 
144     /**
145      * @param replicator The regionserver which has undeleted queue
146      * @param queueId The undeleted queue id
147      * @throws IOException
148      */
149     public void removeQueue(final String replicator, final String queueId) throws IOException {
150       String queueZnodePath = ZKUtil.joinZNode(ZKUtil.joinZNode(this.queuesZNode, replicator),
151         queueId);
152       try {
153         ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
154         if (!replicationPeers.getAllPeerIds().contains(queueInfo.getPeerId())) {
155           ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath);
156           LOG.info("Successfully removed replication queue, replicator: " + replicator
157               + ", queueId: " + queueId);
158         }
159       } catch (KeeperException e) {
160         throw new IOException("Failed to delete queue, replicator: " + replicator + ", queueId: "
161             + queueId);
162       }
163     }
164 
165     /**
166      * @param hfileRefsQueueId The undeleted hfile-refs queue id
167      * @throws IOException
168      */
169     public void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException {
170       String node = ZKUtil.joinZNode(this.hfileRefsZNode, hfileRefsQueueId);
171       try {
172         if (!replicationPeers.getAllPeerIds().contains(hfileRefsQueueId)) {
173           ZKUtil.deleteNodeRecursively(this.zookeeper, node);
174           LOG.info("Successfully removed hfile-refs queue " + hfileRefsQueueId + " from path "
175               + hfileRefsZNode);
176         }
177       } catch (KeeperException e) {
178         throw new IOException("Failed to delete hfile-refs queue " + hfileRefsQueueId
179             + " from path " + hfileRefsZNode);
180       }
181     }
182 
183     String getHfileRefsZNode() {
184       return this.hfileRefsZNode;
185     }
186   }
187 
188   /**
189    * Remove the undeleted replication queue's zk node for removed peers.
190    * @param undeletedQueues replicator with its queueIds for removed peers
191    * @throws IOException
192    */
193   public void removeQueues(final Map<String, List<String>> undeletedQueues) throws IOException {
194     for (Entry<String, List<String>> replicatorAndQueueIds : undeletedQueues.entrySet()) {
195       String replicator = replicatorAndQueueIds.getKey();
196       for (String queueId : replicatorAndQueueIds.getValue()) {
197         queueDeletor.removeQueue(replicator, queueId);
198       }
199     }
200   }
201 
202   /**
203    * Remove the undeleted hfile-refs queue's zk node for removed peers.
204    * @param undeletedHFileRefsQueues replicator with its undeleted queueIds for removed peers in
205    *          hfile-refs queue
206    * @throws IOException
207    */
208   public void removeHFileRefsQueues(final Set<String> undeletedHFileRefsQueues) throws IOException {
209     for (String hfileRefsQueueId : undeletedHFileRefsQueues) {
210       queueDeletor.removeHFileRefsQueue(hfileRefsQueueId);
211     }
212   }
213 }