View Javadoc

1   /**
2    * Copyright 2009 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.zookeeper;
21  
22  import java.io.BufferedReader;
23  import java.io.IOException;
24  import java.io.InputStreamReader;
25  import java.io.PrintWriter;
26  import java.net.Socket;
27  import java.util.ArrayList;
28  import java.util.HashMap;
29  import java.util.HashSet;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.Properties;
33  import java.util.Set;
34  import java.util.concurrent.locks.Lock;
35  import java.util.concurrent.locks.ReentrantLock;
36  
37  import org.apache.commons.logging.Log;
38  import org.apache.commons.logging.LogFactory;
39  import org.apache.hadoop.conf.Configuration;
40  import org.apache.hadoop.hbase.HConstants;
41  import org.apache.hadoop.hbase.HServerAddress;
42  import org.apache.hadoop.hbase.HServerInfo;
43  import org.apache.hadoop.hbase.util.Bytes;
44  import org.apache.zookeeper.CreateMode;
45  import org.apache.zookeeper.KeeperException;
46  import org.apache.zookeeper.WatchedEvent;
47  import org.apache.zookeeper.Watcher;
48  import org.apache.zookeeper.ZooKeeper;
49  import org.apache.zookeeper.ZooDefs.Ids;
50  import org.apache.zookeeper.ZooKeeper.States;
51  import org.apache.zookeeper.data.Stat;
52  
53  /**
54   * Wraps a ZooKeeper instance and adds HBase specific functionality.
55   *
56   * This class provides methods to:
57   * - read/write/delete the root region location in ZooKeeper.
58   * - set/check out of safe mode flag.
59   *
60   * ------------------------------------------
61   * The following STATIC ZNodes are created:
62   * ------------------------------------------
63   * - parentZNode     : All the HBase directories are hosted under this parent
64   *                     node, default = "/hbase"
65   * - rsZNode         : This is the directory where the RS's create ephemeral
66   *                     nodes. The master watches these nodes, and their expiry
67   *                     indicates RS death. The default location is "/hbase/rs"
68   *
69   * ------------------------------------------
70   * The following DYNAMIC ZNodes are created:
71   * ------------------------------------------
72   * - rootRegionZNode     : Specifies the RS hosting root.
73   * - masterElectionZNode : ZNode used for election of the primary master when
74   *                         there are secondaries. All the masters race to write
75   *                         their addresses into this location, the one that
76   *                         succeeds is the primary. Others block.
77   * - clusterStateZNode   : Determines if the cluster is running. Its default
78   *                         location is "/hbase/shutdown". It always has a value
79   *                         of "up". If present with the valus, cluster is up
80   *                         and running. If deleted, the cluster is shutting
81   *                         down.
82   * - rgnsInTransitZNode  : All the nodes under this node are names of regions
83   *                         in transition. The first byte of the data for each
84   *                         of these nodes is the event type. This is used to
85   *                         deserialize the rest of the data.
86   */
87  public class ZooKeeperWrapper implements Watcher {
88    protected static final Log LOG = LogFactory.getLog(ZooKeeperWrapper.class);
89  
90    // instances of the watcher
91    private static Map<String,ZooKeeperWrapper> INSTANCES =
92      new HashMap<String,ZooKeeperWrapper>();
93    // lock for ensuring a singleton per instance type
94    private static Lock createLock = new ReentrantLock();
95    // name of this instance
96    private String instanceName;
97  
98    // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
99    private static final char ZNODE_PATH_SEPARATOR = '/';
100 
101   private String quorumServers = null;
102   private final int sessionTimeout;
103   private ZooKeeper zooKeeper;
104 
105   /*
106    * All the HBase directories are hosted under this parent
107    */
108   public final String parentZNode;
109   /*
110    * Specifies the RS hosting root
111    */
112   private final String rootRegionZNode;
113   /*
114    * This is the directory where the RS's create ephemeral nodes. The master
115    * watches these nodes, and their expiry indicates RS death.
116    */
117   private final String rsZNode;
118   /*
119    * ZNode used for election of the primary master when there are secondaries.
120    */
121   private final String masterElectionZNode;
122   /*
123    * State of the cluster - if up and running or shutting down
124    */
125   public final String clusterStateZNode;
126 
127   private List<Watcher> listeners = new ArrayList<Watcher>();
128 
129   // return the singleton given the name of the instance
130   public static ZooKeeperWrapper getInstance(Configuration conf, String name) {
131     name = getZookeeperClusterKey(conf, name);
132     return INSTANCES.get(name);
133   }
134   // creates only one instance
135   public static ZooKeeperWrapper createInstance(Configuration conf, String name) {
136     if (getInstance(conf, name) != null) {
137       return getInstance(conf, name);
138     }
139     ZooKeeperWrapper.createLock.lock();
140     try {
141       if (getInstance(conf, name) == null) {
142         try {
143           String fullname = getZookeeperClusterKey(conf, name);
144           ZooKeeperWrapper instance = new ZooKeeperWrapper(conf, fullname);
145           INSTANCES.put(fullname, instance);
146         }
147         catch (Exception e) {
148           LOG.error("<" + name + ">" + "Error creating a ZooKeeperWrapper " + e);
149         }
150       }
151     }
152     finally {
153       createLock.unlock();
154     }
155     return getInstance(conf, name);
156   }
157 
158   /**
159    * Create a ZooKeeperWrapper. The Zookeeper wrapper listens to all messages
160    * from Zookeeper, and notifies all the listeners about all the messages. Any
161    * component can subscribe to these messages by adding itself as a listener,
162    * and remove itself from being a listener.
163    *
164    * @param conf HBaseConfiguration to read settings from.
165    * @throws IOException If a connection error occurs.
166    */
167   private ZooKeeperWrapper(Configuration conf, String instanceName)
168   throws IOException {
169     this.instanceName = instanceName;
170     Properties properties = HQuorumPeer.makeZKProps(conf);
171     quorumServers = HQuorumPeer.getZKQuorumServersString(properties);
172     if (quorumServers == null) {
173       throw new IOException("Could not read quorum servers from " +
174                             HConstants.ZOOKEEPER_CONFIG_NAME);
175     }
176     sessionTimeout = conf.getInt("zookeeper.session.timeout", 60 * 1000);
177     reconnectToZk();
178 
179     parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
180 
181     String rootServerZNodeName = conf.get("zookeeper.znode.rootserver", "root-region-server");
182     String rsZNodeName         = conf.get("zookeeper.znode.rs", "rs");
183     String masterAddressZNodeName = conf.get("zookeeper.znode.master", "master");
184     String stateZNodeName      = conf.get("zookeeper.znode.state", "shutdown");
185     String regionsInTransitZNodeName = conf.get("zookeeper.znode.regionInTransition", "UNASSIGNED");
186 
187     rootRegionZNode     = getZNode(parentZNode, rootServerZNodeName);
188     rsZNode             = getZNode(parentZNode, rsZNodeName);
189     masterElectionZNode = getZNode(parentZNode, masterAddressZNodeName);
190     clusterStateZNode   = getZNode(parentZNode, stateZNodeName);
191   }
192 
193   public void reconnectToZk() throws IOException {
194     try {
195       LOG.info("Reconnecting to zookeeper");
196       if(zooKeeper != null) {
197         zooKeeper.close();
198         LOG.debug("<" + instanceName + ">" + "Closed existing zookeeper client");
199       }
200       zooKeeper = new ZooKeeper(quorumServers, sessionTimeout, this);
201       LOG.debug("<" + instanceName + ">" + "Connected to zookeeper again");
202     } catch (IOException e) {
203       LOG.error("<" + instanceName + ">" + "Failed to create ZooKeeper object: " + e);
204       throw new IOException(e);
205     } catch (InterruptedException e) {
206       LOG.error("<" + instanceName + ">" + "Error closing ZK connection: " + e);
207       throw new IOException(e);
208     }
209   }
210 
211   public synchronized void registerListener(Watcher watcher) {
212     listeners.add(watcher);
213   }
214 
215   public synchronized void unregisterListener(Watcher watcher) {
216     listeners.remove(watcher);
217   }
218 
219   /**
220    * This is the primary ZK watcher
221    * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatchedEvent)
222    */
223   @Override
224   public synchronized void process(WatchedEvent event) {
225     for(Watcher w : listeners) {
226       try {
227         w.process(event);
228       } catch (Throwable t) {
229         LOG.error("<"+instanceName+">" + "ZK updates listener threw an exception in process()", t);
230       }
231     }
232   }
233 
234   /** @return String dump of everything in ZooKeeper. */
235   @SuppressWarnings({"ConstantConditions"})
236   public String dump() {
237     StringBuilder sb = new StringBuilder();
238     sb.append("\nHBase tree in ZooKeeper is rooted at ").append(parentZNode);
239     sb.append("\n  Cluster up? ").append(exists(clusterStateZNode, true));
240     sb.append("\n  Master address: ").append(readMasterAddress(null));
241     sb.append("\n  Region server holding ROOT: ").append(readRootRegionLocation());
242     sb.append("\n  Region servers:");
243     for (HServerAddress address : scanRSDirectory()) {
244       sb.append("\n    - ").append(address);
245     }
246     sb.append("\n  Quorum Server Statistics:");
247     String[] servers = quorumServers.split(",");
248     for (String server : servers) {
249       sb.append("\n    - ").append(server);
250       try {
251         String[] stat = getServerStats(server);
252         for (String s : stat) {
253           sb.append("\n        ").append(s);
254         }
255       } catch (Exception e) {
256         sb.append("\n        ERROR: ").append(e.getMessage());
257       }
258     }
259     return sb.toString();
260   }
261 
262   /**
263    * Gets the statistics from the given server. Uses a 1 minute timeout.
264    *
265    * @param server  The server to get the statistics from.
266    * @return The array of response strings.
267    * @throws IOException When the socket communication fails.
268    */
269   public String[] getServerStats(String server)
270   throws IOException {
271     return getServerStats(server, 60 * 1000);
272   }
273 
274   /**
275    * Gets the statistics from the given server.
276    *
277    * @param server  The server to get the statistics from.
278    * @param timeout  The socket timeout to use.
279    * @return The array of response strings.
280    * @throws IOException When the socket communication fails.
281    */
282   public String[] getServerStats(String server, int timeout)
283   throws IOException {
284     String[] sp = server.split(":");
285     Socket socket = new Socket(sp[0],
286       sp.length > 1 ? Integer.parseInt(sp[1]) : 2181);
287     socket.setSoTimeout(timeout);
288     PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
289     BufferedReader in = new BufferedReader(new InputStreamReader(
290       socket.getInputStream()));
291     out.println("stat");
292     out.flush();
293     ArrayList<String> res = new ArrayList<String>();
294     while (true) {
295       String line = in.readLine();
296       if (line != null) res.add(line);
297       else break;
298     }
299     socket.close();
300     return res.toArray(new String[res.size()]);
301   }
302 
303   public boolean exists(String znode, boolean watch) {
304     try {
305       return zooKeeper.exists(getZNode(parentZNode, znode), watch?this:null) != null;
306     } catch (KeeperException.SessionExpiredException e) {
307       // if the session has expired try to reconnect to ZK, then perform query
308       try {
309         // TODO: ZK-REFACTOR: We should not reconnect - we should just quit and restart.
310         reconnectToZk();
311         return zooKeeper.exists(getZNode(parentZNode, znode), watch?this:null) != null;
312       } catch (IOException e1) {
313         LOG.error("Error reconnecting to zookeeper", e1);
314         throw new RuntimeException("Error reconnecting to zookeeper", e1);
315       } catch (KeeperException e1) {
316         LOG.error("Error reading after reconnecting to zookeeper", e1);
317         throw new RuntimeException("Error reading after reconnecting to zookeeper", e1);
318       } catch (InterruptedException e1) {
319         LOG.error("Error reading after reconnecting to zookeeper", e1);
320         throw new RuntimeException("Error reading after reconnecting to zookeeper", e1);
321       }
322     } catch (KeeperException e) {
323       return false;
324     } catch (InterruptedException e) {
325       return false;
326     }
327   }
328 
329   /** @return ZooKeeper used by this wrapper. */
330   public ZooKeeper getZooKeeper() {
331     return zooKeeper;
332   }
333 
334   /**
335    * This is for testing KeeperException.SessionExpiredException.
336    * See HBASE-1232.
337    * @return long session ID of this ZooKeeper session.
338    */
339   public long getSessionID() {
340     return zooKeeper.getSessionId();
341   }
342 
343   /**
344    * This is for testing KeeperException.SessionExpiredException.
345    * See HBASE-1232.
346    * @return byte[] password of this ZooKeeper session.
347    */
348   public byte[] getSessionPassword() {
349     return zooKeeper.getSessionPasswd();
350   }
351 
352   /** @return host:port list of quorum servers. */
353   public String getQuorumServers() {
354     return quorumServers;
355   }
356 
357   /** @return true if currently connected to ZooKeeper, false otherwise. */
358   public boolean isConnected() {
359     return zooKeeper.getState() == States.CONNECTED;
360   }
361 
362   /**
363    * Read location of server storing root region.
364    * @return HServerAddress pointing to server serving root region or null if
365    *         there was a problem reading the ZNode.
366    */
367   public HServerAddress readRootRegionLocation() {
368     return readAddress(rootRegionZNode, null);
369   }
370 
371   /**
372    * Read address of master server.
373    * @return HServerAddress of master server.
374    * @throws IOException if there's a problem reading the ZNode.
375    */
376   public HServerAddress readMasterAddressOrThrow() throws IOException {
377     return readAddressOrThrow(masterElectionZNode, null);
378   }
379 
380   /**
381    * Read master address and set a watch on it.
382    * @param watcher Watcher to set on master address ZNode if not null.
383    * @return HServerAddress of master or null if there was a problem reading the
384    *         ZNode. The watcher is set only if the result is not null.
385    */
386   public HServerAddress readMasterAddress(Watcher watcher) {
387     return readAddress(masterElectionZNode, watcher);
388   }
389 
390   /**
391    * Watch the state of the cluster, up or down
392    * @param watcher Watcher to set on cluster state node
393    */
394   public void setClusterStateWatch(Watcher watcher) {
395     try {
396       zooKeeper.exists(clusterStateZNode, watcher == null ? this : watcher);
397     } catch (InterruptedException e) {
398       LOG.warn("<" + instanceName + ">" + "Failed to check on ZNode " + clusterStateZNode, e);
399     } catch (KeeperException e) {
400       LOG.warn("<" + instanceName + ">" + "Failed to check on ZNode " + clusterStateZNode, e);
401     }
402   }
403 
404   /**
405    * Set the cluster state, up or down
406    * @param up True to write the node, false to delete it
407    * @return true if it worked, else it's false
408    */
409   public boolean setClusterState(boolean up) {
410     if (!ensureParentExists(clusterStateZNode)) {
411       return false;
412     }
413     try {
414       if(up) {
415         byte[] data = Bytes.toBytes("up");
416         zooKeeper.create(clusterStateZNode, data,
417             Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
418         LOG.debug("<" + instanceName + ">" + "State node wrote in ZooKeeper");
419       } else {
420         zooKeeper.delete(clusterStateZNode, -1);
421         LOG.debug("<" + instanceName + ">" + "State node deleted in ZooKeeper");
422       }
423       return true;
424     } catch (InterruptedException e) {
425       LOG.warn("<" + instanceName + ">" + "Failed to set state node in ZooKeeper", e);
426     } catch (KeeperException e) {
427       if(e.code() == KeeperException.Code.NODEEXISTS) {
428         LOG.debug("<" + instanceName + ">" + "State node exists.");
429       } else {
430         LOG.warn("<" + instanceName + ">" + "Failed to set state node in ZooKeeper", e);
431       }
432     }
433 
434     return false;
435   }
436 
437   /**
438    * Set a watcher on the master address ZNode. The watcher will be set unless
439    * an exception occurs with ZooKeeper.
440    * @param watcher Watcher to set on master address ZNode.
441    * @return true if watcher was set, false otherwise.
442    */
443   public boolean watchMasterAddress(Watcher watcher) {
444     try {
445       zooKeeper.exists(masterElectionZNode, watcher);
446     } catch (KeeperException e) {
447       LOG.warn("<" + instanceName + ">" + "Failed to set watcher on ZNode " + masterElectionZNode, e);
448       return false;
449     } catch (InterruptedException e) {
450       LOG.warn("<" + instanceName + ">" + "Failed to set watcher on ZNode " + masterElectionZNode, e);
451       return false;
452     }
453     LOG.debug("<" + instanceName + ">" + "Set watcher on master address ZNode " + masterElectionZNode);
454     return true;
455   }
456   
457   /**
458    * @return true if zookeeper has a master address.
459    */
460   public boolean masterAddressExists() {
461     return checkExistenceOf(masterElectionZNode);
462   }
463 
464   private HServerAddress readAddress(String znode, Watcher watcher) {
465     try {
466       LOG.debug("<" + instanceName + ">" + "Trying to read " + znode);
467       return readAddressOrThrow(znode, watcher);
468     } catch (IOException e) {
469       LOG.debug("<" + instanceName + ">" + "Failed to read " + e.getMessage());
470       return null;
471     }
472   }
473 
474   private HServerAddress readAddressOrThrow(String znode, Watcher watcher) throws IOException {
475     byte[] data;
476     try {
477       data = zooKeeper.getData(znode, watcher, null);
478     } catch (InterruptedException e) {
479       throw new IOException(e);
480     } catch (KeeperException e) {
481       throw new IOException(e);
482     }
483 
484     String addressString = Bytes.toString(data);
485     LOG.debug("<" + instanceName + ">" + "Read ZNode " + znode + " got " + addressString);
486     return new HServerAddress(addressString);
487   }
488 
489   /**
490    * Make sure this znode exists by creating it if it's missing
491    * @param znode full path to znode
492    * @return true if it works
493    */
494   public boolean ensureExists(final String znode) {
495     try {
496       Stat stat = zooKeeper.exists(znode, false);
497       if (stat != null) {
498         return true;
499       }
500       zooKeeper.create(znode, new byte[0],
501                        Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
502       LOG.debug("<" + instanceName + ">" + "Created ZNode " + znode);
503       return true;
504     } catch (KeeperException.NodeExistsException e) {
505       return true;      // ok, move on.
506     } catch (KeeperException.NoNodeException e) {
507       return ensureParentExists(znode) && ensureExists(znode);
508     } catch (KeeperException e) {
509       LOG.warn("<" + instanceName + ">" + "Failed to create " + znode +
510         " -- check quorum servers, currently=" + this.quorumServers, e);
511     } catch (InterruptedException e) {
512       LOG.warn("<" + instanceName + ">" + "Failed to create " + znode +
513         " -- check quorum servers, currently=" + this.quorumServers, e);
514     }
515     return false;
516   }
517 
518   private boolean ensureParentExists(final String znode) {
519     int index = znode.lastIndexOf(ZNODE_PATH_SEPARATOR);
520     if (index <= 0) {   // Parent is root, which always exists.
521       return true;
522     }
523     return ensureExists(znode.substring(0, index));
524   }
525 
526   /**
527    * Delete ZNode containing root region location.
528    * @return true if operation succeeded, false otherwise.
529    */
530   public boolean deleteRootRegionLocation()  {
531     if (!ensureParentExists(rootRegionZNode)) {
532       return false;
533     }
534 
535     try {
536       deleteZNode(rootRegionZNode);
537       return true;
538     } catch (KeeperException.NoNodeException e) {
539       return true;    // ok, move on.
540     } catch (KeeperException e) {
541       LOG.warn("<" + instanceName + ">" + "Failed to delete " + rootRegionZNode + ": " + e);
542     } catch (InterruptedException e) {
543       LOG.warn("<" + instanceName + ">" + "Failed to delete " + rootRegionZNode + ": " + e);
544     }
545 
546     return false;
547   }
548 
549   /**
550    * Unrecursive deletion of specified znode
551    * @param znode
552    * @throws KeeperException
553    * @throws InterruptedException
554    */
555   public void deleteZNode(String znode)
556       throws KeeperException, InterruptedException {
557     deleteZNode(znode, false);
558   }
559 
560   /**
561    * Optionnally recursive deletion of specified znode
562    * @param znode
563    * @param recursive
564    * @throws KeeperException
565    * @throws InterruptedException
566    */
567   public void deleteZNode(String znode, boolean recursive)
568     throws KeeperException, InterruptedException {
569     if (recursive) {
570       LOG.info("<" + instanceName + ">" + "deleteZNode get children for " + znode);
571       List<String> znodes = this.zooKeeper.getChildren(znode, false);
572       if (znodes != null && znodes.size() > 0) {
573         for (String child : znodes) {
574           String childFullPath = getZNode(znode, child);
575           LOG.info("<" + instanceName + ">" + "deleteZNode recursive call " + childFullPath);
576           this.deleteZNode(childFullPath, true);
577         }
578       }
579     }
580     this.zooKeeper.delete(znode, -1);
581     LOG.debug("<" + instanceName + ">" + "Deleted ZNode " + znode);
582   }
583 
584   private boolean createRootRegionLocation(String address) {
585     byte[] data = Bytes.toBytes(address);
586     try {
587       zooKeeper.create(rootRegionZNode, data, Ids.OPEN_ACL_UNSAFE,
588                        CreateMode.PERSISTENT);
589       LOG.debug("<" + instanceName + ">" + "Created ZNode " + rootRegionZNode + " with data " + address);
590       return true;
591     } catch (KeeperException e) {
592       LOG.warn("<" + instanceName + ">" + "Failed to create root region in ZooKeeper: " + e);
593     } catch (InterruptedException e) {
594       LOG.warn("<" + instanceName + ">" + "Failed to create root region in ZooKeeper: " + e);
595     }
596 
597     return false;
598   }
599 
600   private boolean updateRootRegionLocation(String address) {
601     byte[] data = Bytes.toBytes(address);
602     try {
603       zooKeeper.setData(rootRegionZNode, data, -1);
604       LOG.debug("<" + instanceName + ">" + "SetData of ZNode " + rootRegionZNode + " with " + address);
605       return true;
606     } catch (KeeperException e) {
607       LOG.warn("<" + instanceName + ">" + "Failed to set root region location in ZooKeeper: " + e);
608     } catch (InterruptedException e) {
609       LOG.warn("<" + instanceName + ">" + "Failed to set root region location in ZooKeeper: " + e);
610     }
611 
612     return false;
613   }
614 
615   /**
616    * Write root region location to ZooKeeper. If address is null, delete ZNode.
617    * containing root region location.
618    * @param address HServerAddress to write to ZK.
619    * @return true if operation succeeded, false otherwise.
620    */
621   public boolean writeRootRegionLocation(HServerAddress address) {
622     if (address == null) {
623       return deleteRootRegionLocation();
624     }
625 
626     if (!ensureParentExists(rootRegionZNode)) {
627       return false;
628     }
629 
630     String addressString = address.toString();
631 
632     if (checkExistenceOf(rootRegionZNode)) {
633       return updateRootRegionLocation(addressString);
634     }
635 
636     return createRootRegionLocation(addressString);
637   }
638 
639   /**
640    * Write address of master to ZooKeeper.
641    * @param address HServerAddress of master.
642    * @return true if operation succeeded, false otherwise.
643    */
644   public boolean writeMasterAddress(final HServerAddress address) {
645     LOG.debug("<" + instanceName + ">" + "Writing master address " + address.toString() + " to znode " + masterElectionZNode);
646     if (!ensureParentExists(masterElectionZNode)) {
647       return false;
648     }
649     LOG.debug("<" + instanceName + ">" + "Znode exists : " + masterElectionZNode);
650 
651     String addressStr = address.toString();
652     byte[] data = Bytes.toBytes(addressStr);
653     try {
654       zooKeeper.create(masterElectionZNode, data, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
655       LOG.debug("<" + instanceName + ">" + "Wrote master address " + address + " to ZooKeeper");
656       return true;
657     } catch (InterruptedException e) {
658       LOG.warn("<" + instanceName + ">" + "Failed to write master address " + address + " to ZooKeeper", e);
659     } catch (KeeperException e) {
660       LOG.warn("<" + instanceName + ">" + "Failed to write master address " + address + " to ZooKeeper", e);
661     }
662 
663     return false;
664   }
665 
666   /**
667    * Write in ZK this RS startCode and address.
668    * Ensures that the full path exists.
669    * @param info The RS info
670    * @return true if the location was written, false if it failed
671    */
672   public boolean writeRSLocation(HServerInfo info) {
673     ensureExists(rsZNode);
674     byte[] data = Bytes.toBytes(info.getServerAddress().toString());
675     String znode = joinPath(rsZNode, info.getServerName());
676     try {
677       zooKeeper.create(znode, data, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
678       LOG.debug("<" + instanceName + ">" + "Created ZNode " + znode
679           + " with data " + info.getServerAddress().toString());
680       return true;
681     } catch (KeeperException e) {
682       LOG.warn("<" + instanceName + ">" + "Failed to create " + znode + " znode in ZooKeeper: " + e);
683     } catch (InterruptedException e) {
684       LOG.warn("<" + instanceName + ">" + "Failed to create " + znode + " znode in ZooKeeper: " + e);
685     }
686     return false;
687   }
688 
689   /**
690    * Update the RS address and set a watcher on the znode
691    * @param info The RS info
692    * @param watcher The watcher to put on the znode
693    * @return true if the update is done, false if it failed
694    */
695   public boolean updateRSLocationGetWatch(HServerInfo info, Watcher watcher) {
696     byte[] data = Bytes.toBytes(info.getServerAddress().toString());
697     String znode = rsZNode + ZNODE_PATH_SEPARATOR + info.getServerName();
698     try {
699       zooKeeper.setData(znode, data, -1);
700       LOG.debug("<" + instanceName + ">" + "Updated ZNode " + znode
701           + " with data " + info.getServerAddress().toString());
702       zooKeeper.getData(znode, watcher, null);
703       return true;
704     } catch (KeeperException e) {
705       LOG.warn("<" + instanceName + ">" + "Failed to update " + znode + " znode in ZooKeeper: " + e);
706     } catch (InterruptedException e) {
707       LOG.warn("<" + instanceName + ">" + "Failed to update " + znode + " znode in ZooKeeper: " + e);
708     }
709 
710     return false;
711   }
712 
713   /**
714    * Scans the regions servers directory
715    * @return A list of server addresses
716    */
717   public List<HServerAddress> scanRSDirectory() {
718     return scanAddressDirectory(rsZNode, null);
719   }
720 
721   /**
722    * Scans the regions servers directory and sets a watch on each znode
723    * @param watcher a watch to use for each znode
724    * @return A list of server addresses
725    */
726   public List<HServerAddress> scanRSDirectory(Watcher watcher) {
727     return scanAddressDirectory(rsZNode, watcher);
728   }
729 
730   /**
731    * Method used to make sure the region server directory is empty.
732    *
733    */
734   public void clearRSDirectory() {
735     try {
736       List<String> nodes = zooKeeper.getChildren(rsZNode, false);
737       for (String node : nodes) {
738         LOG.debug("<" + instanceName + ">" + "Deleting node: " + node);
739         zooKeeper.delete(joinPath(this.rsZNode, node), -1);
740       }
741     } catch (KeeperException e) {
742       LOG.warn("<" + instanceName + ">" + "Failed to delete " + rsZNode + " znodes in ZooKeeper: " + e);
743     } catch (InterruptedException e) {
744       LOG.warn("<" + instanceName + ">" + "Failed to delete " + rsZNode + " znodes in ZooKeeper: " + e);
745     }
746   }
747 
748   /**
749    * @return the number of region server znodes in the RS directory
750    */
751   public int getRSDirectoryCount() {
752     Stat stat = null;
753     try {
754       stat = zooKeeper.exists(rsZNode, false);
755     } catch (KeeperException e) {
756       LOG.warn("Problem getting stats for " + rsZNode, e);
757     } catch (InterruptedException e) {
758       LOG.warn("Problem getting stats for " + rsZNode, e);
759     }
760     return (stat != null) ? stat.getNumChildren() : 0;
761   }
762 
763   private boolean checkExistenceOf(String path) {
764     Stat stat = null;
765     try {
766       stat = zooKeeper.exists(path, false);
767     } catch (KeeperException e) {
768       LOG.warn("<" + instanceName + ">" + "checking existence of " + path, e);
769     } catch (InterruptedException e) {
770       LOG.warn("<" + instanceName + ">" + "checking existence of " + path, e);
771     }
772 
773     return stat != null;
774   }
775 
776   /**
777    * Close this ZooKeeper session.
778    */
779   public void close() {
780     try {
781       zooKeeper.close();
782       INSTANCES.remove(instanceName);
783       LOG.debug("<" + instanceName + ">" + "Closed connection with ZooKeeper; " + this.rootRegionZNode);
784     } catch (InterruptedException e) {
785       LOG.warn("<" + instanceName + ">" + "Failed to close connection with ZooKeeper");
786     }
787   }
788 
789   public String getZNode(String parentZNode, String znodeName) {
790     return znodeName.charAt(0) == ZNODE_PATH_SEPARATOR ?
791         znodeName : joinPath(parentZNode, znodeName);
792   }
793 
794   public String getZNodePathForHBase(String znodeName) {
795     return getZNode(parentZNode, znodeName);
796   }
797 
798   private String joinPath(String parent, String child) {
799     return parent + ZNODE_PATH_SEPARATOR + child;
800   }
801 
802   /**
803    * Get the path of the masterElectionZNode
804    * @return the path to masterElectionZNode
805    */
806   public String getMasterElectionZNode() {
807     return masterElectionZNode;
808   }
809 
810   /**
811    * Get the path of the parent ZNode
812    * @return path of that znode
813    */
814   public String getParentZNode() {
815     return parentZNode;
816   }
817 
818   /**
819    * Scan a directory of address data.
820    * @param znode The parent node
821    * @param watcher The watcher to put on the found znodes, if not null
822    * @return The directory contents
823    */
824   public List<HServerAddress> scanAddressDirectory(String znode,
825       Watcher watcher) {
826     List<HServerAddress> list = new ArrayList<HServerAddress>();
827     List<String> nodes = this.listZnodes(znode);
828     if(nodes == null) {
829       return list;
830     }
831     for (String node : nodes) {
832       String path = joinPath(znode, node);
833       list.add(readAddress(path, watcher));
834     }
835     return list;
836   }
837 
838   /**
839    * List all znodes in the specified path
840    * @param znode path to list
841    * @return a list of all the znodes
842    */
843   public List<String> listZnodes(String znode) {
844     return listZnodes(znode, this);
845   }
846 
847   /**
848    * List all znodes in the specified path and set a watcher on each
849    * @param znode path to list
850    * @param watcher watch to set, can be null
851    * @return a list of all the znodes
852    */
853   public List<String> listZnodes(String znode, Watcher watcher) {
854     List<String> nodes = null;
855     if (watcher == null) {
856       watcher = this;
857     }
858     try {
859       if (checkExistenceOf(znode)) {
860         nodes = zooKeeper.getChildren(znode, watcher);
861         for (String node : nodes) {
862           getDataAndWatch(znode, node, watcher);
863         }
864       }
865     } catch (KeeperException e) {
866       LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e);
867     } catch (InterruptedException e) {
868       LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e);
869     }
870     return nodes;
871   }
872 
873   public byte[] getData(String parentZNode, String znode) {
874     return getDataAndWatch(parentZNode, znode, null);
875   }
876 
877   public byte[] getDataAndWatch(String parentZNode,
878                                 String znode, Watcher watcher) {
879     byte[] data = null;
880     try {
881       String path = joinPath(parentZNode, znode);
882       // TODO: ZK-REFACTOR: remove existance check?
883       if (checkExistenceOf(path)) {
884         data = zooKeeper.getData(path, watcher, null);
885       }
886     } catch (KeeperException e) {
887       LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e);
888     } catch (InterruptedException e) {
889       LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e);
890     }
891     return data;
892   }
893 
894   /**
895    * Write a znode and fail if it already exists
896    * @param parentPath parent path to the new znode
897    * @param child name of the znode
898    * @param strData data to insert
899    * @throws InterruptedException
900    * @throws KeeperException
901    */
902   public void writeZNode(String parentPath, String child, String strData)
903       throws InterruptedException, KeeperException {
904     writeZNode(parentPath, child, strData, false);
905   }
906 
907 
908   /**
909    * Write (and optionally over-write) a znode
910    * @param parentPath parent path to the new znode
911    * @param child name of the znode
912    * @param strData data to insert
913    * @param failOnWrite true if an exception should be returned if the znode
914    * already exists, false if it should be overwritten
915    * @throws InterruptedException
916    * @throws KeeperException
917    */
918   public void writeZNode(String parentPath, String child, String strData,
919       boolean failOnWrite) throws InterruptedException, KeeperException {
920     String path = joinPath(parentPath, child);
921     if (!ensureExists(parentPath)) {
922       LOG.error("<" + instanceName + ">" + "unable to ensure parent exists: " + parentPath);
923     }
924     byte[] data = Bytes.toBytes(strData);
925     Stat stat = this.zooKeeper.exists(path, false);
926     if (failOnWrite || stat == null) {
927       this.zooKeeper.create(path, data,
928           Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
929       LOG.debug("<" + instanceName + ">" + "Created " + path + " with data " + strData);
930     } else {
931       this.zooKeeper.setData(path, data, -1);
932       LOG.debug("<" + instanceName + ">" + "Updated " + path + " with data " + strData);
933     }
934   }
935 
936   /**
937    * Get the key to the ZK ensemble for this configuration without
938    * adding a name at the end
939    * @param conf Configuration to use to build the key
940    * @return ensemble key without a name
941    */
942   public static String getZookeeperClusterKey(Configuration conf) {
943     return getZookeeperClusterKey(conf, null);
944   }
945 
946   /**
947    * Get the key to the ZK ensemble for this configuration and append
948    * a name at the end
949    * @param conf Configuration to use to build the key
950    * @param name Name that should be appended at the end if not empty or null
951    * @return ensemble key with a name (if any)
952    */
953   public static String getZookeeperClusterKey(Configuration conf, String name) {
954     String quorum = conf.get(HConstants.ZOOKEEPER_QUORUM.replaceAll(
955         "[\\t\\n\\x0B\\f\\r]", ""));
956     StringBuilder builder = new StringBuilder(quorum);
957     builder.append(":");
958     builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
959     if (name != null && !name.isEmpty()) {
960       builder.append(",");
961       builder.append(name);
962     }
963     return builder.toString();
964   }
965 
966   /**
967    * Get the path of this region server's znode
968    * @return path to znode
969    */
970   public String getRsZNode() {
971     return this.rsZNode;
972   }
973 
974   public void deleteZNode(String zNodeName, int version) {
975     String fullyQualifiedZNodeName = getZNode(parentZNode, zNodeName);
976     try
977     {
978       zooKeeper.delete(fullyQualifiedZNodeName, version);
979     }
980     catch (InterruptedException e)
981     {
982       LOG.warn("<" + instanceName + ">" + "Failed to delete ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e);
983     }
984     catch (KeeperException e)
985     {
986       LOG.warn("<" + instanceName + ">" + "Failed to delete ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e);
987     }
988   }
989 
990   public String createZNodeIfNotExists(String zNodeName) {
991     return createZNodeIfNotExists(zNodeName, null, CreateMode.PERSISTENT, true);
992   }
993 
994   public void watchZNode(String zNodeName) {
995     String fullyQualifiedZNodeName = getZNode(parentZNode, zNodeName);
996 
997     try {
998       zooKeeper.exists(fullyQualifiedZNodeName, this);
999       zooKeeper.getData(fullyQualifiedZNodeName, this, null);
1000       zooKeeper.getChildren(fullyQualifiedZNodeName, this);
1001     } catch (InterruptedException e) {
1002       LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e);
1003     } catch (KeeperException e) {
1004       LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e);
1005     }
1006   }
1007 
1008   public String createZNodeIfNotExists(String zNodeName, byte[] data, CreateMode createMode, boolean watch) {
1009     String fullyQualifiedZNodeName = getZNode(parentZNode, zNodeName);
1010 
1011     if (!ensureParentExists(fullyQualifiedZNodeName)) {
1012       return null;
1013     }
1014 
1015     try {
1016       // create the znode
1017       zooKeeper.create(fullyQualifiedZNodeName, data, Ids.OPEN_ACL_UNSAFE, createMode);
1018       LOG.debug("<" + instanceName + ">" + "Created ZNode " + fullyQualifiedZNodeName + " in ZooKeeper");
1019       // watch the znode for deletion, data change, creation of children
1020       if(watch) {
1021         watchZNode(zNodeName);
1022       }
1023       return fullyQualifiedZNodeName;
1024     } catch (InterruptedException e) {
1025       LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e);
1026     } catch (KeeperException e) {
1027       LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e);
1028     }
1029 
1030     return null;
1031   }
1032 
1033   public byte[] readZNode(String znodeName, Stat stat) throws IOException {
1034     byte[] data;
1035     try {
1036       String fullyQualifiedZNodeName = getZNode(parentZNode, znodeName);
1037       data = zooKeeper.getData(fullyQualifiedZNodeName, this, stat);
1038     } catch (InterruptedException e) {
1039       throw new IOException(e);
1040     } catch (KeeperException e) {
1041       throw new IOException(e);
1042     }
1043     return data;
1044   }
1045 
1046   // TODO: perhaps return the version number from this write?
1047   public boolean writeZNode(String znodeName, byte[] data, int version, boolean watch) throws IOException {
1048       try {
1049         String fullyQualifiedZNodeName = getZNode(parentZNode, znodeName);
1050         zooKeeper.setData(fullyQualifiedZNodeName, data, version);
1051         if(watch) {
1052           zooKeeper.getData(fullyQualifiedZNodeName, this, null);
1053         }
1054         return true;
1055       } catch (InterruptedException e) {
1056         LOG.warn("<" + instanceName + ">" + "Failed to write data to ZooKeeper", e);
1057         throw new IOException(e);
1058       } catch (KeeperException e) {
1059         LOG.warn("<" + instanceName + ">" + "Failed to write data to ZooKeeper", e);
1060         throw new IOException(e);
1061       }
1062     }
1063 
1064 
1065   public static class ZNodePathAndData {
1066     private String zNodePath;
1067     private byte[] data;
1068 
1069     public ZNodePathAndData(String zNodePath, byte[] data) {
1070       this.zNodePath = zNodePath;
1071       this.data = data;
1072     }
1073 
1074     public String getzNodePath() {
1075       return zNodePath;
1076     }
1077     public byte[] getData() {
1078       return data;
1079     }
1080 
1081   }
1082 }