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