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