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