View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.replication;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Collections;
25  import java.util.HashMap;
26  import java.util.List;
27  import java.util.Map;
28  import java.util.SortedMap;
29  import java.util.SortedSet;
30  import java.util.TreeMap;
31  import java.util.TreeSet;
32  import java.util.UUID;
33  import java.util.concurrent.atomic.AtomicBoolean;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.hbase.Abortable;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.Server;
41  import org.apache.hadoop.hbase.ServerName;
42  import org.apache.hadoop.hbase.replication.regionserver.Replication;
43  import org.apache.hadoop.hbase.util.Bytes;
44  import org.apache.hadoop.hbase.zookeeper.ClusterId;
45  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
46  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
47  import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
48  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
49  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
50  import org.apache.zookeeper.KeeperException;
51  import org.apache.zookeeper.KeeperException.AuthFailedException;
52  import org.apache.zookeeper.KeeperException.ConnectionLossException;
53  import org.apache.zookeeper.KeeperException.SessionExpiredException;
54  
55  /**
56   * This class serves as a helper for all things related to zookeeper in
57   * replication.
58   * <p/>
59   * The layout looks something like this under zookeeper.znode.parent for the
60   * master cluster:
61   * <p/>
62   *
63   * <pre>
64   * replication/
65   *  state      {contains true or false}
66   *  clusterId  {contains a byte}
67   *  peers/
68   *    1/   {contains a full cluster address}
69   *      peer-state  {contains ENABLED or DISABLED}
70   *    2/
71   *    ...
72   *  rs/ {lists all RS that replicate}
73   *    startcode1/ {lists all peer clusters}
74   *      1/ {lists hlogs to process}
75   *        10.10.1.76%3A53488.123456789 {contains nothing or a position}
76   *        10.10.1.76%3A53488.123456790
77   *        ...
78   *      2/
79   *      ...
80   *    startcode2/
81   *    ...
82   * </pre>
83   */
84  public class ReplicationZookeeper {
85    private static final Log LOG =
86      LogFactory.getLog(ReplicationZookeeper.class);
87    // Name of znode we use to lock when failover
88    private final static String RS_LOCK_ZNODE = "lock";
89  
90    // Values of znode which stores state of a peer
91    public static enum PeerState {
92      ENABLED, DISABLED
93    };
94  
95    // Our handle on zookeeper
96    private final ZooKeeperWatcher zookeeper;
97    // Map of peer clusters keyed by their id
98    private Map<String, ReplicationPeer> peerClusters;
99    // Path to the root replication znode
100   private String replicationZNode;
101   // Path to the peer clusters znode
102   private String peersZNode;
103   // Path to the znode that contains all RS that replicates
104   private String rsZNode;
105   // Path to this region server's name under rsZNode
106   private String rsServerNameZnode;
107   // Name node if the replicationState znode
108   private String replicationStateNodeName;
109   // Name of zk node which stores peer state
110   private String peerStateNodeName;
111   private final Configuration conf;
112   // Is this cluster replicating at the moment?
113   private AtomicBoolean replicating;
114   // The key to our own cluster
115   private String ourClusterKey;
116   // Abortable
117   private Abortable abortable;
118   private ReplicationStatusTracker statusTracker;
119 
120   /**
121    * Constructor used by clients of replication (like master and HBase clients)
122    * @param conf  conf to use
123    * @param zk    zk connection to use
124    * @throws IOException
125    */
126   public ReplicationZookeeper(final Abortable abortable, final Configuration conf,
127                               final ZooKeeperWatcher zk)
128     throws KeeperException {
129 
130     this.conf = conf;
131     this.zookeeper = zk;
132     this.replicating = new AtomicBoolean();
133     setZNodes(abortable);
134   }
135 
136   /**
137    * Constructor used by region servers, connects to the peer cluster right away.
138    *
139    * @param server
140    * @param replicating    atomic boolean to start/stop replication
141    * @throws IOException
142    * @throws KeeperException 
143    */
144   public ReplicationZookeeper(final Server server, final AtomicBoolean replicating)
145   throws IOException, KeeperException {
146     this.abortable = server;
147     this.zookeeper = server.getZooKeeper();
148     this.conf = server.getConfiguration();
149     this.replicating = replicating;
150     setZNodes(server);
151 
152     this.peerClusters = new HashMap<String, ReplicationPeer>();
153     ZKUtil.createWithParents(this.zookeeper,
154         ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
155     this.rsServerNameZnode = ZKUtil.joinZNode(rsZNode, server.getServerName().toString());
156     ZKUtil.createWithParents(this.zookeeper, this.rsServerNameZnode);
157     connectExistingPeers();
158   }
159 
160   private void setZNodes(Abortable abortable) throws KeeperException {
161     String replicationZNodeName =
162         conf.get("zookeeper.znode.replication", "replication");
163     String peersZNodeName =
164         conf.get("zookeeper.znode.replication.peers", "peers");
165     this.peerStateNodeName = conf.get(
166         "zookeeper.znode.replication.peers.state", "peer-state");
167     this.replicationStateNodeName =
168         conf.get("zookeeper.znode.replication.state", "state");
169     String rsZNodeName =
170         conf.get("zookeeper.znode.replication.rs", "rs");
171     this.ourClusterKey = ZKUtil.getZooKeeperClusterKey(this.conf);
172     this.replicationZNode =
173       ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
174     this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
175     ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
176     this.rsZNode = ZKUtil.joinZNode(replicationZNode, rsZNodeName);
177     ZKUtil.createWithParents(this.zookeeper, this.rsZNode);
178 
179     // Set a tracker on replicationStateNodeNode
180     this.statusTracker =
181         new ReplicationStatusTracker(this.zookeeper, abortable);
182     statusTracker.start();
183     readReplicationStateZnode();
184   }
185 
186   private void connectExistingPeers() throws IOException, KeeperException {
187     List<String> znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
188     if (znodes != null) {
189       for (String z : znodes) {
190         connectToPeer(z);
191       }
192     }
193   }
194 
195   /**
196    * List this cluster's peers' IDs
197    * @return list of all peers' identifiers
198    */
199   public List<String> listPeersIdsAndWatch() {
200     List<String> ids = null;
201     try {
202       ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode);
203     } catch (KeeperException e) {
204       this.abortable.abort("Cannot get the list of peers ", e);
205     }
206     return ids;
207   }
208 
209   /**
210    * Map of this cluster's peers for display.
211    * @return A map of peer ids to peer cluster keys
212    */
213   public Map<String,String> listPeers() {
214     Map<String,String> peers = new TreeMap<String,String>();
215     List<String> ids = null;
216     try {
217       ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
218       for (String id : ids) {
219         peers.put(id, Bytes.toString(ZKUtil.getData(this.zookeeper,
220             ZKUtil.joinZNode(this.peersZNode, id))));
221       }
222     } catch (KeeperException e) {
223       this.abortable.abort("Cannot get the list of peers ", e);
224     }
225     return peers;
226   }
227   /**
228    * Returns all region servers from given peer
229    *
230    * @param peerClusterId (byte) the cluster to interrogate
231    * @return addresses of all region servers
232    */
233   public List<ServerName> getSlavesAddresses(String peerClusterId) {
234     if (this.peerClusters.size() == 0) {
235       return Collections.emptyList();
236     }
237     ReplicationPeer peer = this.peerClusters.get(peerClusterId);
238     if (peer == null) {
239       return Collections.emptyList();
240     }
241     
242     List<ServerName> addresses;
243     try {
244       addresses = fetchSlavesAddresses(peer.getZkw());
245     } catch (KeeperException ke) {
246       if (LOG.isDebugEnabled()) {
247         LOG.debug("Fetch salves addresses failed.", ke);
248       }
249       reconnectPeer(ke, peer);
250       addresses = Collections.emptyList();
251     }
252     peer.setRegionServers(addresses);
253     return peer.getRegionServers();
254   }
255 
256   /**
257    * Get the list of all the region servers from the specified peer
258    * @param zkw zk connection to use
259    * @return list of region server addresses or an empty list if the slave
260    * is unavailable
261    */
262   private List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
263     throws KeeperException {
264     return listChildrenAndGetAsServerNames(zkw, zkw.rsZNode);
265   }
266 
267   /**
268    * Lists the children of the specified znode, retrieving the data of each
269    * child as a server address.
270    *
271    * Used to list the currently online regionservers and their addresses.
272    *
273    * Sets no watches at all, this method is best effort.
274    *
275    * Returns an empty list if the node has no children.  Returns null if the
276    * parent node itself does not exist.
277    *
278    * @param zkw zookeeper reference
279    * @param znode node to get children of as addresses
280    * @return list of data of children of specified znode, empty if no children,
281    *         null if parent does not exist
282    * @throws KeeperException if unexpected zookeeper exception
283    */
284   public static List<ServerName> listChildrenAndGetAsServerNames(
285       ZooKeeperWatcher zkw, String znode)
286   throws KeeperException {
287     List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
288     if(children == null) {
289       return Collections.emptyList();
290     }
291     List<ServerName> addresses = new ArrayList<ServerName>(children.size());
292     for (String child : children) {
293       addresses.add(ServerName.parseServerName(child));
294     }
295     return addresses;
296   }
297 
298   /**
299    * This method connects this cluster to another one and registers it
300    * in this region server's replication znode
301    * @param peerId id of the peer cluster
302    * @throws KeeperException 
303    */
304   public boolean connectToPeer(String peerId)
305       throws IOException, KeeperException {
306     if (peerClusters == null) {
307       return false;
308     }
309     if (this.peerClusters.containsKey(peerId)) {
310       return false;
311     }
312     ReplicationPeer peer = getPeer(peerId);
313     if (peer == null) {
314       return false;
315     }
316     this.peerClusters.put(peerId, peer);
317     ZKUtil.createWithParents(this.zookeeper, ZKUtil.joinZNode(
318         this.rsServerNameZnode, peerId));
319     LOG.info("Added new peer cluster " + peer.getClusterKey());
320     return true;
321   }
322 
323   /**
324    * Helper method to connect to a peer
325    * @param peerId peer's identifier
326    * @return object representing the peer
327    * @throws IOException
328    * @throws KeeperException
329    */
330   public ReplicationPeer getPeer(String peerId) throws IOException, KeeperException{
331     String znode = ZKUtil.joinZNode(this.peersZNode, peerId);
332     byte [] data = ZKUtil.getData(this.zookeeper, znode);
333     String otherClusterKey = Bytes.toString(data);
334     if (this.ourClusterKey.equals(otherClusterKey)) {
335       LOG.debug("Not connecting to " + peerId + " because it's us");
336       return null;
337     }
338     // Construct the connection to the new peer
339     Configuration otherConf = new Configuration(this.conf);
340     try {
341       ZKUtil.applyClusterKeyToConf(otherConf, otherClusterKey);
342     } catch (IOException e) {
343       LOG.error("Can't get peer because:", e);
344       return null;
345     }
346 
347     ReplicationPeer peer = new ReplicationPeer(otherConf, peerId,
348         otherClusterKey);
349     peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId));
350     return peer;
351   }
352 
353   /**
354    * Set the new replication state for this cluster
355    * @param newState
356    */
357   public void setReplicating(boolean newState) throws KeeperException {
358     ZKUtil.createWithParents(this.zookeeper,
359         ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
360     ZKUtil.setData(this.zookeeper,
361         ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName),
362         Bytes.toBytes(Boolean.toString(newState)));
363   }
364 
365   /**
366    * Remove the peer from zookeeper. which will trigger the watchers on every
367    * region server and close their sources
368    * @param id
369    * @throws IllegalArgumentException Thrown when the peer doesn't exist
370    */
371   public void removePeer(String id) throws IOException {
372     try {
373       if (!peerExists(id)) {
374         throw new IllegalArgumentException("Cannot remove inexisting peer");
375       }
376       ZKUtil.deleteNodeRecursively(this.zookeeper,
377           ZKUtil.joinZNode(this.peersZNode, id));
378     } catch (KeeperException e) {
379       throw new IOException("Unable to remove a peer", e);
380     }
381   }
382 
383   /**
384    * Add a new peer to this cluster
385    * @param id peer's identifier
386    * @param clusterKey ZK ensemble's addresses, client port and root znode
387    * @throws IllegalArgumentException Thrown when the peer doesn't exist
388    * @throws IllegalStateException Thrown when a peer already exists, since
389    *         multi-slave isn't supported yet.
390    */
391   public void addPeer(String id, String clusterKey) throws IOException {
392     try {
393       if (peerExists(id)) {
394         throw new IllegalArgumentException("Cannot add existing peer");
395       }
396       ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
397       ZKUtil.createAndWatch(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id),
398         Bytes.toBytes(clusterKey));
399       // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
400       // peer-state znode. This happens while adding a peer.
401       // The peer state data is set as "ENABLED" by default.
402       ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, getPeerStateNode(id),
403         Bytes.toBytes(PeerState.ENABLED.name()));
404     } catch (KeeperException e) {
405       throw new IOException("Unable to add peer", e);
406     }
407   }
408 
409   private boolean peerExists(String id) throws KeeperException {
410     return ZKUtil.checkExists(this.zookeeper,
411           ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
412   }
413 
414   /**
415    * Enable replication to the peer
416    *
417    * @param id peer's identifier
418    * @throws IllegalArgumentException
419    *           Thrown when the peer doesn't exist
420    */
421   public void enablePeer(String id) throws IOException {
422     changePeerState(id, PeerState.ENABLED);
423     LOG.info("peer " + id + " is enabled");
424   }
425 
426   /**
427    * Disable replication to the peer
428    *
429    * @param id peer's identifier
430    * @throws IllegalArgumentException
431    *           Thrown when the peer doesn't exist
432    */
433   public void disablePeer(String id) throws IOException {
434     changePeerState(id, PeerState.DISABLED);
435     LOG.info("peer " + id + " is disabled");
436   }
437 
438   private void changePeerState(String id, PeerState state) throws IOException {
439     try {
440       if (!peerExists(id)) {
441         throw new IllegalArgumentException("peer " + id + " is not registered");
442       }
443       String peerStateZNode = getPeerStateNode(id);
444       if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
445         ZKUtil.setData(this.zookeeper, peerStateZNode,
446           Bytes.toBytes(state.name()));
447       } else {
448         ZKUtil.createAndWatch(zookeeper, peerStateZNode,
449             Bytes.toBytes(state.name()));
450       }
451       LOG.info("state of the peer " + id + " changed to " + state.name());
452     } catch (KeeperException e) {
453       throw new IOException("Unable to change state of the peer " + id, e);
454     }
455   }
456 
457   /**
458    * Get state of the peer. This method checks the state by connecting to ZK.
459    *
460    * @param id peer's identifier
461    * @return current state of the peer
462    */
463   public PeerState getPeerState(String id) throws KeeperException {
464     byte[] peerStateBytes = ZKUtil
465         .getData(this.zookeeper, getPeerStateNode(id));
466     return PeerState.valueOf(Bytes.toString(peerStateBytes));
467   }
468 
469   /**
470    * Check whether the peer is enabled or not. This method checks the atomic
471    * boolean of ReplicationPeer locally.
472    *
473    * @param id peer identifier
474    * @return true if the peer is enabled, otherwise false
475    * @throws IllegalArgumentException
476    *           Thrown when the peer doesn't exist
477    */
478   public boolean getPeerEnabled(String id) {
479     if (!this.peerClusters.containsKey(id)) {
480       throw new IllegalArgumentException("peer " + id + " is not registered");
481     }
482     return this.peerClusters.get(id).getPeerEnabled().get();
483   }
484 
485   private String getPeerStateNode(String id) {
486     return ZKUtil.joinZNode(this.peersZNode,
487         ZKUtil.joinZNode(id, this.peerStateNodeName));
488   }
489 
490   /**
491    * This reads the state znode for replication and sets the atomic boolean
492    */
493   private void readReplicationStateZnode() {
494     try {
495       this.replicating.set(getReplication());
496       LOG.info("Replication is now " + (this.replicating.get()?
497         "started" : "stopped"));
498     } catch (KeeperException e) {
499       this.abortable.abort("Failed getting data on from " + getRepStateNode(), e);
500     }
501   }
502 
503   /**
504    * Get the replication status of this cluster. If the state znode doesn't
505    * exist it will also create it and set it true.
506    * @return returns true when it's enabled, else false
507    * @throws KeeperException
508    */
509   public boolean getReplication() throws KeeperException {
510     byte [] data = this.statusTracker.getData(false);
511     if (data == null || data.length == 0) {
512       setReplicating(true);
513       return true;
514     }
515     return Boolean.parseBoolean(Bytes.toString(data));
516   }
517 
518   private String getRepStateNode() {
519     return ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName);
520   }
521 
522   /**
523    * Add a new log to the list of hlogs in zookeeper
524    * @param filename name of the hlog's znode
525    * @param peerId name of the cluster's znode
526    */
527   public void addLogToList(String filename, String peerId)
528     throws KeeperException {
529     String znode = ZKUtil.joinZNode(this.rsServerNameZnode, peerId);
530     znode = ZKUtil.joinZNode(znode, filename);
531     ZKUtil.createWithParents(this.zookeeper, znode);
532   }
533 
534   /**
535    * Remove a log from the list of hlogs in zookeeper
536    * @param filename name of the hlog's znode
537    * @param clusterId name of the cluster's znode
538    */
539   public void removeLogFromList(String filename, String clusterId) {
540     try {
541       String znode = ZKUtil.joinZNode(rsServerNameZnode, clusterId);
542       znode = ZKUtil.joinZNode(znode, filename);
543       ZKUtil.deleteNode(this.zookeeper, znode);
544     } catch (KeeperException e) {
545       this.abortable.abort("Failed remove from list", e);
546     }
547   }
548 
549   /**
550    * Set the current position of the specified cluster in the current hlog
551    * @param filename filename name of the hlog's znode
552    * @param clusterId clusterId name of the cluster's znode
553    * @param position the position in the file
554    * @throws IOException
555    */
556   public void writeReplicationStatus(String filename, String clusterId,
557       long position) {
558     try {
559       String znode = ZKUtil.joinZNode(this.rsServerNameZnode, clusterId);
560       znode = ZKUtil.joinZNode(znode, filename);
561       // Why serialize String of Long and note Long as bytes?
562       ZKUtil.setData(this.zookeeper, znode,
563         Bytes.toBytes(Long.toString(position)));
564     } catch (KeeperException e) {
565       this.abortable.abort("Writing replication status", e);
566     }
567   }
568 
569   /**
570    * Get a list of all the other region servers in this cluster
571    * and set a watch
572    * @return a list of server nanes
573    */
574   public List<String> getRegisteredRegionServers() {
575     List<String> result = null;
576     try {
577       result = ZKUtil.listChildrenAndWatchThem(
578           this.zookeeper, this.zookeeper.rsZNode);
579     } catch (KeeperException e) {
580       this.abortable.abort("Get list of registered region servers", e);
581     }
582     return result;
583   }
584 
585   /**
586    * Get the list of the replicators that have queues, they can be alive, dead
587    * or simply from a previous run
588    * @return a list of server names
589    */
590   public List<String> getListOfReplicators() {
591     List<String> result = null;
592     try {
593       result = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZNode);
594     } catch (KeeperException e) {
595       this.abortable.abort("Get list of replicators", e);
596     }
597     return result;
598   }
599 
600   /**
601    * Get the list of peer clusters for the specified server names
602    * @param rs server names of the rs
603    * @return a list of peer cluster
604    */
605   public List<String> getListPeersForRS(String rs) {
606     String znode = ZKUtil.joinZNode(rsZNode, rs);
607     List<String> result = null;
608     try {
609       result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
610     } catch (KeeperException e) {
611       this.abortable.abort("Get list of peers for rs", e);
612     }
613     return result;
614   }
615 
616   /**
617    * Get the list of hlogs for the specified region server and peer cluster
618    * @param rs server names of the rs
619    * @param id peer cluster
620    * @return a list of hlogs
621    */
622   public List<String> getListHLogsForPeerForRS(String rs, String id) {
623     String znode = ZKUtil.joinZNode(rsZNode, rs);
624     znode = ZKUtil.joinZNode(znode, id);
625     List<String> result = null;
626     try {
627       result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
628     } catch (KeeperException e) {
629       this.abortable.abort("Get list of hlogs for peer", e);
630     }
631     return result;
632   }
633 
634   /**
635    * Checks if the provided znode is the same as this region server's
636    * @param znode to check
637    * @return if this is this rs's znode
638    */
639   public boolean isThisOurZnode(String znode) {
640     String otherRs = ZKUtil.joinZNode(this.rsZNode, znode);
641     return otherRs.equals(rsServerNameZnode);
642   }
643 
644   /**
645    * Try to set a lock in another server's znode.
646    * @param znode the server names of the other server
647    * @return true if the lock was acquired, false in every other cases
648    */
649   public boolean lockOtherRS(String znode) {
650     try {
651       String parent = ZKUtil.joinZNode(this.rsZNode, znode);
652       String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
653       ZKUtil.createAndWatch(this.zookeeper, p, Bytes.toBytes(rsServerNameZnode));
654     } catch (KeeperException e) {
655       // This exception will pop up if the znode under which we're trying to
656       // create the lock is already deleted by another region server, meaning
657       // that the transfer already occurred.
658       // NoNode => transfer is done and znodes are already deleted
659       // NodeExists => lock znode already created by another RS
660       if (e instanceof KeeperException.NoNodeException ||
661           e instanceof KeeperException.NodeExistsException) {
662         LOG.info("Won't transfer the queue," +
663             " another RS took care of it because of: " + e.getMessage());
664       } else {
665         LOG.info("Failed lock other rs", e);
666       }
667       return false;
668     }
669     return true;
670   }
671 
672   /**
673    * It "atomically" copies all the hlogs queues from another region server and returns them all
674    * sorted per peer cluster (appended with the dead server's znode).
675    * @param znode
676    * @return HLog queues sorted per peer cluster
677    */
678   public SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
679     SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
680     String deadRSZnodePath = ZKUtil.joinZNode(rsZNode, znode);// hbase/replication/rs/deadrs
681     List<String> peerIdsToProcess = null;
682     List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
683     try {
684       peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
685       if (peerIdsToProcess == null) return queues; // node already processed
686       for (String peerId : peerIdsToProcess) {
687         String newPeerId = peerId + "-" + znode;
688         String newPeerZnode = ZKUtil.joinZNode(this.rsServerNameZnode, newPeerId);
689         // check the logs queue for the old peer cluster
690         String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
691         List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
692         if (hlogs == null || hlogs.size() == 0) {
693           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
694           continue; // empty log queue.
695         }
696         // create the new cluster znode
697         SortedSet<String> logQueue = new TreeSet<String>();
698         queues.put(newPeerId, logQueue);
699         ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
700         listOfOps.add(op);
701         // get the offset of the logs and set it to new znodes
702         for (String hlog : hlogs) {
703           String oldHlogZnode = ZKUtil.joinZNode(oldClusterZnode, hlog);
704           byte[] logOffset = ZKUtil.getData(this.zookeeper, oldHlogZnode);
705           LOG.debug("Creating " + hlog + " with data " + Bytes.toString(logOffset));
706           String newLogZnode = ZKUtil.joinZNode(newPeerZnode, hlog);
707           listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
708           // add ops for deleting
709           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldHlogZnode));
710           logQueue.add(hlog);
711         }
712         // add delete op for peer
713         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
714       }
715       // add delete op for dead rs
716       listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
717       LOG.debug(" The multi list size is: " + listOfOps.size());
718       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
719       LOG.info("Atomically moved the dead regionserver logs. ");
720     } catch (KeeperException e) {
721       // Multi call failed; it looks like some other regionserver took away the logs.
722       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
723       queues.clear();
724     }
725     return queues;
726   }
727 
728   /**
729    * This methods copies all the hlogs queues from another region server
730    * and returns them all sorted per peer cluster (appended with the dead
731    * server's znode)
732    * @param znode server names to copy
733    * @return all hlogs for all peers of that cluster, null if an error occurred
734    */
735   public SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
736     // TODO this method isn't atomic enough, we could start copying and then
737     // TODO fail for some reason and we would end up with znodes we don't want.
738     SortedMap<String,SortedSet<String>> queues =
739         new TreeMap<String,SortedSet<String>>();
740     try {
741       String nodePath = ZKUtil.joinZNode(rsZNode, znode);
742       List<String> clusters =
743         ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
744       // We have a lock znode in there, it will count as one.
745       if (clusters == null || clusters.size() <= 1) {
746         return queues;
747       }
748       // The lock isn't a peer cluster, remove it
749       clusters.remove(RS_LOCK_ZNODE);
750       for (String cluster : clusters) {
751         // We add the name of the recovered RS to the new znode, we can even
752         // do that for queues that were recovered 10 times giving a znode like
753         // number-startcode-number-otherstartcode-number-anotherstartcode-etc
754         String newCluster = cluster+"-"+znode;
755         String newClusterZnode = ZKUtil.joinZNode(rsServerNameZnode, newCluster);
756         String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
757         List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
758         // That region server didn't have anything to replicate for this cluster
759         if (hlogs == null || hlogs.size() == 0) {
760           continue;
761         }
762         ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
763             HConstants.EMPTY_BYTE_ARRAY);
764         SortedSet<String> logQueue = new TreeSet<String>();
765         queues.put(newCluster, logQueue);
766         for (String hlog : hlogs) {
767           String z = ZKUtil.joinZNode(clusterPath, hlog);
768           byte [] position = ZKUtil.getData(this.zookeeper, z);
769           LOG.debug("Creating " + hlog + " with data " + Bytes.toString(position));
770           String child = ZKUtil.joinZNode(newClusterZnode, hlog);
771           ZKUtil.createAndWatch(this.zookeeper, child, position);
772           logQueue.add(hlog);
773         }
774       }
775     } catch (KeeperException e) {
776       this.abortable.abort("Copy queues from rs", e);
777     }
778     return queues;
779   }
780 
781   /**
782    * Delete a complete queue of hlogs
783    * @param peerZnode znode of the peer cluster queue of hlogs to delete
784    */
785   public void deleteSource(String peerZnode, boolean closeConnection) {
786     try {
787       ZKUtil.deleteNodeRecursively(this.zookeeper,
788           ZKUtil.joinZNode(rsServerNameZnode, peerZnode));
789       if (closeConnection) {
790         this.peerClusters.get(peerZnode).getZkw().close();
791         this.peerClusters.remove(peerZnode);
792       }
793     } catch (KeeperException e) {
794       this.abortable.abort("Failed delete of " + peerZnode, e);
795     }
796   }
797 
798   /**
799    * Recursive deletion of all znodes in specified rs' znode
800    * @param znode
801    */
802   public void deleteRsQueues(String znode) {
803     String fullpath = ZKUtil.joinZNode(rsZNode, znode);
804     try {
805       List<String> clusters =
806         ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
807       for (String cluster : clusters) {
808         // We'll delete it later
809         if (cluster.equals(RS_LOCK_ZNODE)) {
810           continue;
811         }
812         String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
813         ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
814       }
815       // Finish cleaning up
816       ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
817     } catch (KeeperException e) {
818       if (e instanceof KeeperException.NoNodeException ||
819           e instanceof KeeperException.NotEmptyException) {
820         // Testing a special case where another region server was able to
821         // create a lock just after we deleted it, but then was also able to
822         // delete the RS znode before us or its lock znode is still there.
823         if (e.getPath().equals(fullpath)) {
824           return;
825         }
826       }
827       this.abortable.abort("Failed delete of " + znode, e);
828     }
829   }
830 
831   /**
832    * Delete this cluster's queues
833    */
834   public void deleteOwnRSZNode() {
835     try {
836       ZKUtil.deleteNodeRecursively(this.zookeeper,
837           this.rsServerNameZnode);
838     } catch (KeeperException e) {
839       // if the znode is already expired, don't bother going further
840       if (e instanceof KeeperException.SessionExpiredException) {
841         return;
842       }
843       this.abortable.abort("Failed delete of " + this.rsServerNameZnode, e);
844     }
845   }
846 
847   /**
848    * Get the position of the specified hlog in the specified peer znode
849    * @param peerId znode of the peer cluster
850    * @param hlog name of the hlog
851    * @return the position in that hlog
852    * @throws KeeperException 
853    */
854   public long getHLogRepPosition(String peerId, String hlog)
855   throws KeeperException {
856     String clusterZnode = ZKUtil.joinZNode(rsServerNameZnode, peerId);
857     String znode = ZKUtil.joinZNode(clusterZnode, hlog);
858     String data = Bytes.toString(ZKUtil.getData(this.zookeeper, znode));
859     return data == null || data.length() == 0 ? 0 : Long.parseLong(data);
860   }
861 
862   /**
863    * Returns the UUID of the provided peer id. Should a connection loss or session
864    * expiration happen, the ZK handler will be reopened once and if it still doesn't
865    * work then it will bail and return null.
866    * @param peerId the peer's ID that will be converted into a UUID
867    * @return a UUID or null if there's a ZK connection issue
868    */
869   public UUID getPeerUUID(String peerId) {
870     ReplicationPeer peer = getPeerClusters().get(peerId);
871     UUID peerUUID = null;
872     try {
873       peerUUID = getUUIDForCluster(peer.getZkw());
874     } catch (KeeperException ke) {
875       reconnectPeer(ke, peer);
876     }
877     return peerUUID;
878   }
879 
880   /**
881    * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions
882    * @param zkw watcher connected to an ensemble
883    * @return the UUID read from zookeeper
884    * @throws KeeperException
885    */
886   public UUID getUUIDForCluster(ZooKeeperWatcher zkw) throws KeeperException {
887     return UUID.fromString(ClusterId.readClusterIdZNode(zkw));
888   }
889 
890   private void reconnectPeer(KeeperException ke, ReplicationPeer peer) {
891     if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException
892         || ke instanceof AuthFailedException) {
893       LOG.warn(
894         "Lost the ZooKeeper connection for peer " + peer.getClusterKey(),
895         ke);
896       try {
897         peer.reloadZkWatcher();
898       } catch(IOException io) {
899         LOG.warn(
900           "Creation of ZookeeperWatcher failed for peer "
901             + peer.getClusterKey(), io);
902       }
903     }
904   }
905 
906   public void registerRegionServerListener(ZooKeeperListener listener) {
907     this.zookeeper.registerListener(listener);
908   }
909 
910   /**
911    * Get a map of all peer clusters
912    * @return map of peer cluster keyed by id
913    */
914   public Map<String, ReplicationPeer> getPeerClusters() {
915     return this.peerClusters;
916   }
917 
918   /**
919    * Extracts the znode name of a peer cluster from a ZK path
920    * @param fullPath Path to extract the id from
921    * @return the id or an empty string if path is invalid
922    */
923   public static String getZNodeName(String fullPath) {
924     String[] parts = fullPath.split("/");
925     return parts.length > 0 ? parts[parts.length-1] : "";
926   }
927 
928   /**
929    * Get this cluster's zk connection
930    * @return zk connection
931    */
932   public ZooKeeperWatcher getZookeeperWatcher() {
933     return this.zookeeper;
934   }
935 
936 
937   /**
938    * Get the full path to the peers' znode
939    * @return path to peers in zk
940    */
941   public String getPeersZNode() {
942     return peersZNode;
943   }
944 
945   /**
946    * Tracker for status of the replication
947    */
948   public class ReplicationStatusTracker extends ZooKeeperNodeTracker {
949     public ReplicationStatusTracker(ZooKeeperWatcher watcher,
950         Abortable abortable) {
951       super(watcher, getRepStateNode(), abortable);
952     }
953 
954     @Override
955     public synchronized void nodeDataChanged(String path) {
956       if (path.equals(node)) {
957         super.nodeDataChanged(path);
958         readReplicationStateZnode();
959       }
960     }
961   }
962 }