View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.zookeeper;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.HashSet;
25  import java.util.List;
26  import java.util.Set;
27  import java.util.concurrent.CopyOnWriteArrayList;
28  import java.util.concurrent.CountDownLatch;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.hbase.Abortable;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
36  import org.apache.hadoop.hbase.util.Threads;
37  import org.apache.zookeeper.KeeperException;
38  import org.apache.zookeeper.WatchedEvent;
39  import org.apache.zookeeper.Watcher;
40  import org.apache.zookeeper.ZooDefs;
41  import org.apache.zookeeper.data.ACL;
42  
43  /**
44   * Acts as the single ZooKeeper Watcher.  One instance of this is instantiated
45   * for each Master, RegionServer, and client process.
46   *
47   * <p>This is the only class that implements {@link Watcher}.  Other internal
48   * classes which need to be notified of ZooKeeper events must register with
49   * the local instance of this watcher via {@link #registerListener}.
50   *
51   * <p>This class also holds and manages the connection to ZooKeeper.  Code to
52   * deal with connection related events and exceptions are handled here.
53   */
54  public class ZooKeeperWatcher implements Watcher, Abortable {
55    private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
56  
57    // Identifier for this watcher (for logging only).  It is made of the prefix
58    // passed on construction and the zookeeper sessionid.
59    private String identifier;
60  
61    // zookeeper quorum
62    private String quorum;
63  
64    // zookeeper connection
65    private RecoverableZooKeeper recoverableZooKeeper;
66  
67    // abortable in case of zk failure
68    private Abortable abortable;
69  
70    // listeners to be notified
71    private final List<ZooKeeperListener> listeners =
72      new CopyOnWriteArrayList<ZooKeeperListener>();
73  
74    // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
75    // negotiation to complete
76    public CountDownLatch saslLatch = new CountDownLatch(1);
77  
78    // set of unassigned nodes watched
79    private Set<String> unassignedNodes = new HashSet<String>();
80  
81    // node names
82  
83    // base znode for this cluster
84    public String baseZNode;
85    // znode containing location of server hosting root region
86    public String rootServerZNode;
87    // znode containing ephemeral nodes of the regionservers
88    public String rsZNode;
89    // znode containing ephemeral nodes of the draining regionservers
90    public String drainingZNode;
91    // znode of currently active master
92    public String masterAddressZNode;
93    // znode containing the current cluster state
94    public String clusterStateZNode;
95    // znode used for region transitioning and assignment
96    public String assignmentZNode;
97    // znode used for table disabling/enabling
98    public String tableZNode;
99    // znode containing the unique cluster ID
100   public String clusterIdZNode;
101   // znode used for log splitting work assignment
102   public String splitLogZNode;
103 
104   // Certain ZooKeeper nodes need to be world-readable
105   public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
106     new ArrayList<ACL>() { {
107       add(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
108       add(new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
109     }};
110 
111   private final Configuration conf;
112 
113   private final Exception constructorCaller;
114 
115   /**
116    * Instantiate a ZooKeeper connection and watcher.
117    * @param descriptor Descriptive string that is added to zookeeper sessionid
118    * and used as identifier for this instance.
119    * @throws IOException
120    * @throws ZooKeeperConnectionException
121    */
122   public ZooKeeperWatcher(Configuration conf, String descriptor,
123       Abortable abortable) throws ZooKeeperConnectionException, IOException {
124     this(conf, descriptor, abortable, false);
125   }
126   /**
127    * Instantiate a ZooKeeper connection and watcher.
128    * @param descriptor Descriptive string that is added to zookeeper sessionid
129    * and used as identifier for this instance.
130    * @throws IOException
131    * @throws ZooKeeperConnectionException
132    */
133   public ZooKeeperWatcher(Configuration conf, String descriptor,
134       Abortable abortable, boolean canCreateBaseZNode)
135   throws IOException, ZooKeeperConnectionException {
136     this.conf = conf;
137     // Capture a stack trace now.  Will print it out later if problem so we can
138     // distingush amongst the myriad ZKWs.
139     try {
140       throw new Exception("ZKW CONSTRUCTOR STACK TRACE FOR DEBUGGING");
141     } catch (Exception e) {
142       this.constructorCaller = e;
143     }
144     this.quorum = ZKConfig.getZKQuorumServersString(conf);
145     // Identifier will get the sessionid appended later below down when we
146     // handle the syncconnect event.
147     this.identifier = descriptor;
148     this.abortable = abortable;
149     setNodeNames(conf);
150     this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, descriptor);
151     if (canCreateBaseZNode) {
152       createBaseZNodes();
153     }
154   }
155 
156   private void createBaseZNodes() throws ZooKeeperConnectionException {
157     try {
158       // Create all the necessary "directories" of znodes
159       ZKUtil.createAndFailSilent(this, baseZNode);
160       ZKUtil.createAndFailSilent(this, assignmentZNode);
161       ZKUtil.createAndFailSilent(this, rsZNode);
162       ZKUtil.createAndFailSilent(this, drainingZNode);
163       ZKUtil.createAndFailSilent(this, tableZNode);
164       ZKUtil.createAndFailSilent(this, splitLogZNode);
165     } catch (KeeperException e) {
166       throw new ZooKeeperConnectionException(
167           prefix("Unexpected KeeperException creating base node"), e);
168     }
169   }
170 
171   private boolean isFinishedRetryingRecoverable(final long finished) {
172     return System.currentTimeMillis() < finished;
173   }
174 
175   @Override
176   public String toString() {
177     return this.identifier;
178   }
179 
180   /**
181    * Adds this instance's identifier as a prefix to the passed <code>str</code>
182    * @param str String to amend.
183    * @return A new string with this instance's identifier as prefix: e.g.
184    * if passed 'hello world', the returned string could be
185    */
186   public String prefix(final String str) {
187     return this.toString() + " " + str;
188   }
189 
190   /**
191    * Set the local variable node names using the specified configuration.
192    */
193   private void setNodeNames(Configuration conf) {
194     baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
195         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
196     rootServerZNode = ZKUtil.joinZNode(baseZNode,
197         conf.get("zookeeper.znode.rootserver", "root-region-server"));
198     rsZNode = ZKUtil.joinZNode(baseZNode,
199         conf.get("zookeeper.znode.rs", "rs"));
200     drainingZNode = ZKUtil.joinZNode(baseZNode,
201         conf.get("zookeeper.znode.draining.rs", "draining"));
202     masterAddressZNode = ZKUtil.joinZNode(baseZNode,
203         conf.get("zookeeper.znode.master", "master"));
204     clusterStateZNode = ZKUtil.joinZNode(baseZNode,
205         conf.get("zookeeper.znode.state", "shutdown"));
206     assignmentZNode = ZKUtil.joinZNode(baseZNode,
207         conf.get("zookeeper.znode.unassigned", "unassigned"));
208     tableZNode = ZKUtil.joinZNode(baseZNode,
209         conf.get("zookeeper.znode.tableEnableDisable", "table"));
210     clusterIdZNode = ZKUtil.joinZNode(baseZNode,
211         conf.get("zookeeper.znode.clusterId", "hbaseid"));
212     splitLogZNode = ZKUtil.joinZNode(baseZNode,
213         conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME));
214   }
215 
216   /**
217    * Register the specified listener to receive ZooKeeper events.
218    * @param listener
219    */
220   public void registerListener(ZooKeeperListener listener) {
221     listeners.add(listener);
222   }
223 
224   /**
225    * Register the specified listener to receive ZooKeeper events and add it as
226    * the first in the list of current listeners.
227    * @param listener
228    */
229   public void registerListenerFirst(ZooKeeperListener listener) {
230     listeners.add(0, listener);
231   }
232 
233   /**
234    * Get the connection to ZooKeeper.
235    * @return connection reference to zookeeper
236    */
237   public RecoverableZooKeeper getRecoverableZooKeeper() {
238     return recoverableZooKeeper;
239   }
240 
241   /**
242    * Get the quorum address of this instance.
243    * @return quorum string of this zookeeper connection instance
244    */
245   public String getQuorum() {
246     return quorum;
247   }
248 
249   /**
250    * Method called from ZooKeeper for events and connection status.
251    * <p>
252    * Valid events are passed along to listeners.  Connection status changes
253    * are dealt with locally.
254    */
255   @Override
256   public void process(WatchedEvent event) {
257     LOG.debug(prefix("Received ZooKeeper Event, " +
258         "type=" + event.getType() + ", " +
259         "state=" + event.getState() + ", " +
260         "path=" + event.getPath()));
261 
262     switch(event.getType()) {
263 
264       // If event type is NONE, this is a connection status change
265       case None: {
266         connectionEvent(event);
267         break;
268       }
269 
270       // Otherwise pass along to the listeners
271 
272       case NodeCreated: {
273         for(ZooKeeperListener listener : listeners) {
274           listener.nodeCreated(event.getPath());
275         }
276         break;
277       }
278 
279       case NodeDeleted: {
280         for(ZooKeeperListener listener : listeners) {
281           listener.nodeDeleted(event.getPath());
282         }
283         break;
284       }
285 
286       case NodeDataChanged: {
287         for(ZooKeeperListener listener : listeners) {
288           listener.nodeDataChanged(event.getPath());
289         }
290         break;
291       }
292 
293       case NodeChildrenChanged: {
294         for(ZooKeeperListener listener : listeners) {
295           listener.nodeChildrenChanged(event.getPath());
296         }
297         break;
298       }
299     }
300   }
301 
302   // Connection management
303 
304   /**
305    * Called when there is a connection-related event via the Watcher callback.
306    * <p>
307    * If Disconnected or Expired, this should shutdown the cluster. But, since
308    * we send a KeeperException.SessionExpiredException along with the abort
309    * call, it's possible for the Abortable to catch it and try to create a new
310    * session with ZooKeeper. This is what the client does in HCM.
311    * <p>
312    * @param event
313    */
314   private void connectionEvent(WatchedEvent event) {
315     switch(event.getState()) {
316       case SyncConnected:
317         // Now, this callback can be invoked before the this.zookeeper is set.
318         // Wait a little while.
319         long finished = System.currentTimeMillis() +
320           this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait", 2000);
321         while (System.currentTimeMillis() < finished) {
322           Threads.sleep(1);
323           if (this.recoverableZooKeeper != null) break;
324         }
325         if (this.recoverableZooKeeper == null) {
326           LOG.error("ZK is null on connection event -- see stack trace " +
327             "for the stack trace when constructor was called on this zkw",
328             this.constructorCaller);
329           throw new NullPointerException("ZK is null");
330         }
331         this.identifier = this.identifier + "-0x" +
332           Long.toHexString(this.recoverableZooKeeper.getSessionId());
333         // Update our identifier.  Otherwise ignore.
334         LOG.debug(this.identifier + " connected");
335         break;
336 
337       case SaslAuthenticated:
338         if (ZKUtil.isSecureZooKeeper(this.conf)) {
339           // We are authenticated, clients can proceed.
340           saslLatch.countDown();
341         }
342         break;
343 
344       case AuthFailed:
345         if (ZKUtil.isSecureZooKeeper(this.conf)) {
346           // We could not be authenticated, but clients should proceed anyway.
347           // Only access to znodes that require SASL authentication will be
348           // denied. The client may never need to access them.
349           saslLatch.countDown();
350         }
351         break;
352 
353       // Abort the server if Disconnected or Expired
354       case Disconnected:
355         LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
356         break;
357 
358       case Expired:
359         if (ZKUtil.isSecureZooKeeper(this.conf)) {
360           // We consider Expired equivalent to AuthFailed for this
361           // connection. Authentication is never going to complete. The
362           // client should proceed to do cleanup.
363           saslLatch.countDown();
364         }
365         String msg = prefix(this.identifier + " received expired from " +
366           "ZooKeeper, aborting");
367         // TODO: One thought is to add call to ZooKeeperListener so say,
368         // ZooKeeperNodeTracker can zero out its data values.
369         if (this.abortable != null) this.abortable.abort(msg,
370             new KeeperException.SessionExpiredException());
371         break;
372     }
373   }
374 
375   /**
376    * Forces a synchronization of this ZooKeeper client connection.
377    * <p>
378    * Executing this method before running other methods will ensure that the
379    * subsequent operations are up-to-date and consistent as of the time that
380    * the sync is complete.
381    * <p>
382    * This is used for compareAndSwap type operations where we need to read the
383    * data of an existing node and delete or transition that node, utilizing the
384    * previously read version and data.  We want to ensure that the version read
385    * is up-to-date from when we begin the operation.
386    */
387   public void sync(String path) {
388     this.recoverableZooKeeper.sync(path, null, null);
389   }
390 
391   /**
392    * Handles KeeperExceptions in client calls.
393    * <p>
394    * This may be temporary but for now this gives one place to deal with these.
395    * <p>
396    * TODO: Currently this method rethrows the exception to let the caller handle
397    * <p>
398    * @param ke
399    * @throws KeeperException
400    */
401   public void keeperException(KeeperException ke)
402   throws KeeperException {
403     LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
404     throw ke;
405   }
406 
407   /**
408    * Handles InterruptedExceptions in client calls.
409    * <p>
410    * This may be temporary but for now this gives one place to deal with these.
411    * <p>
412    * TODO: Currently, this method does nothing.
413    *       Is this ever expected to happen?  Do we abort or can we let it run?
414    *       Maybe this should be logged as WARN?  It shouldn't happen?
415    * <p>
416    * @param ie
417    */
418   public void interruptedException(InterruptedException ie) {
419     LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie);
420     // At least preserver interrupt.
421     Thread.currentThread().interrupt();
422     // no-op
423   }
424 
425   /**
426    * Close the connection to ZooKeeper.
427    * @throws InterruptedException
428    */
429   public void close() {
430     try {
431       if (recoverableZooKeeper != null) {
432         recoverableZooKeeper.close();
433 //        super.close();
434       }
435     } catch (InterruptedException e) {
436     }
437   }
438 
439   public Configuration getConfiguration() {
440     return conf;
441   }
442 
443   @Override
444   public void abort(String why, Throwable e) {
445     this.abortable.abort(why, e);
446   }
447   
448   @Override
449   public boolean isAborted() {
450     return this.abortable.isAborted();
451   }
452 }