View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.zookeeper;
20  
21  import java.io.Closeable;
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.HashMap;
25  import java.util.List;
26  import java.util.Map;
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.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.hbase.Abortable;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.HRegionInfo;
37  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
38  import org.apache.hadoop.security.UserGroupInformation;
39  import org.apache.zookeeper.KeeperException;
40  import org.apache.zookeeper.WatchedEvent;
41  import org.apache.zookeeper.Watcher;
42  import org.apache.zookeeper.ZooDefs;
43  import org.apache.zookeeper.ZooDefs.Ids;
44  import org.apache.zookeeper.ZooDefs.Perms;
45  import org.apache.zookeeper.data.ACL;
46  import org.apache.zookeeper.data.Id;
47  import org.apache.zookeeper.data.Stat;
48  
49  /**
50   * Acts as the single ZooKeeper Watcher.  One instance of this is instantiated
51   * for each Master, RegionServer, and client process.
52   *
53   * <p>This is the only class that implements {@link Watcher}.  Other internal
54   * classes which need to be notified of ZooKeeper events must register with
55   * the local instance of this watcher via {@link #registerListener}.
56   *
57   * <p>This class also holds and manages the connection to ZooKeeper.  Code to
58   * deal with connection related events and exceptions are handled here.
59   */
60  @InterfaceAudience.Private
61  public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
62    private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
63  
64    // Identifier for this watcher (for logging only).  It is made of the prefix
65    // passed on construction and the zookeeper sessionid.
66    private String prefix;
67    private String identifier;
68  
69    // zookeeper quorum
70    private String quorum;
71  
72    // zookeeper connection
73    private final RecoverableZooKeeper recoverableZooKeeper;
74  
75    // abortable in case of zk failure
76    protected Abortable abortable;
77    // Used if abortable is null
78    private boolean aborted = false;
79  
80    // listeners to be notified
81    private final List<ZooKeeperListener> listeners =
82      new CopyOnWriteArrayList<ZooKeeperListener>();
83  
84    // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
85    // negotiation to complete
86    public CountDownLatch saslLatch = new CountDownLatch(1);
87  
88    // node names
89  
90    // base znode for this cluster
91    public String baseZNode;
92    //znodes containing the locations of the servers hosting the meta replicas
93    private Map<Integer,String> metaReplicaZnodes = new HashMap<Integer, String>();
94    // znode containing ephemeral nodes of the regionservers
95    public String rsZNode;
96    // znode containing ephemeral nodes of the draining regionservers
97    public String drainingZNode;
98    // znode of currently active master
99    private String masterAddressZNode;
100   // znode of this master in backup master directory, if not the active master
101   public String backupMasterAddressesZNode;
102   // znode containing the current cluster state
103   public String clusterStateZNode;
104   // znode used for region transitioning and assignment
105   public String assignmentZNode;
106   // znode used for table disabling/enabling
107   public String tableZNode;
108   // znode containing the unique cluster ID
109   public String clusterIdZNode;
110   // znode used for log splitting work assignment
111   public String splitLogZNode;
112   // znode containing the state of the load balancer
113   public String balancerZNode;
114   // znode containing the state of region normalizer
115   private String regionNormalizerZNode;
116   // znode containing the state of all switches, currently there are split and merge child node.
117   private String switchZNode;
118   // znode containing the lock for the tables
119   public String tableLockZNode;
120   // znode containing the state of recovering regions
121   public String recoveringRegionsZNode;
122   // znode containing namespace descriptors
123   public static String namespaceZNode = "namespace";
124 
125   // Certain ZooKeeper nodes need to be world-readable
126   public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
127     new ArrayList<ACL>() { {
128       add(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
129       add(new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
130     }};
131 
132   public final static String META_ZNODE_PREFIX = "meta-region-server";
133 
134   private final Configuration conf;
135 
136   /**
137    * Instantiate a ZooKeeper connection and watcher.
138    * @param identifier string that is passed to RecoverableZookeeper to be used as
139    * identifier for this instance. Use null for default.
140    * @throws IOException
141    * @throws ZooKeeperConnectionException
142    */
143   public ZooKeeperWatcher(Configuration conf, String identifier,
144       Abortable abortable) throws ZooKeeperConnectionException, IOException {
145     this(conf, identifier, abortable, false);
146   }
147 
148   /**
149    * Instantiate a ZooKeeper connection and watcher.
150    * @param conf
151    * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
152    *          this instance. Use null for default.
153    * @param abortable Can be null if there is on error there is no host to abort: e.g. client
154    *          context.
155    * @param canCreateBaseZNode
156    * @throws IOException
157    * @throws ZooKeeperConnectionException
158    */
159   public ZooKeeperWatcher(Configuration conf, String identifier,
160       Abortable abortable, boolean canCreateBaseZNode)
161   throws IOException, ZooKeeperConnectionException {
162     this.conf = conf;
163     this.quorum = ZKConfig.getZKQuorumServersString(conf);
164     this.prefix = identifier;
165     // Identifier will get the sessionid appended later below down when we
166     // handle the syncconnect event.
167     this.identifier = identifier + "0x0";
168     this.abortable = abortable;
169     setNodeNames(conf);
170     PendingWatcher pendingWatcher = new PendingWatcher();
171     this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier);
172     pendingWatcher.prepare(this);
173     if (canCreateBaseZNode) {
174       createBaseZNodes();
175     }
176   }
177 
178   private void createBaseZNodes() throws ZooKeeperConnectionException {
179     try {
180       // Create all the necessary "directories" of znodes
181       ZKUtil.createWithParents(this, baseZNode);
182       if (conf.getBoolean("hbase.assignment.usezk", true)) {
183         ZKUtil.createAndFailSilent(this, assignmentZNode);
184       }
185       ZKUtil.createAndFailSilent(this, rsZNode);
186       ZKUtil.createAndFailSilent(this, drainingZNode);
187       ZKUtil.createAndFailSilent(this, tableZNode);
188       ZKUtil.createAndFailSilent(this, splitLogZNode);
189       ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode);
190       ZKUtil.createAndFailSilent(this, tableLockZNode);
191       ZKUtil.createAndFailSilent(this, recoveringRegionsZNode);
192     } catch (KeeperException e) {
193       throw new ZooKeeperConnectionException(
194           prefix("Unexpected KeeperException creating base node"), e);
195     }
196   }
197 
198   /** Returns whether the znode is supposed to be readable by the client
199    * and DOES NOT contain sensitive information (world readable).*/
200   public boolean isClientReadable(String node) {
201     // Developer notice: These znodes are world readable. DO NOT add more znodes here UNLESS
202     // all clients need to access this data to work. Using zk for sharing data to clients (other
203     // than service lookup case is not a recommended design pattern.
204     return
205         node.equals(baseZNode) ||
206         isAnyMetaReplicaZnode(node) ||
207         node.equals(getMasterAddressZNode()) ||
208         node.equals(clusterIdZNode)||
209         node.equals(rsZNode) ||
210         // /hbase/table and /hbase/table/foo is allowed, /hbase/table-lock is not
211         node.equals(tableZNode) ||
212         node.startsWith(tableZNode + "/");
213   }
214 
215   /**
216    * On master start, we check the znode ACLs under the root directory and set the ACLs properly
217    * if needed. If the cluster goes from an unsecure setup to a secure setup, this step is needed
218    * so that the existing znodes created with open permissions are now changed with restrictive
219    * perms.
220    */
221   public void checkAndSetZNodeAcls() {
222     if (!ZKUtil.isSecureZooKeeper(getConfiguration())) {
223       return;
224     }
225 
226     // Check the base znodes permission first. Only do the recursion if base znode's perms are not
227     // correct.
228     try {
229       List<ACL> actualAcls = recoverableZooKeeper.getAcl(baseZNode, new Stat());
230 
231       if (!isBaseZnodeAclSetup(actualAcls)) {
232         LOG.info("setting znode ACLs");
233         setZnodeAclsRecursive(baseZNode);
234       }
235     } catch(KeeperException.NoNodeException nne) {
236       return;
237     } catch(InterruptedException ie) {
238       interruptedException(ie);
239     } catch (IOException|KeeperException e) {
240       LOG.warn("Received exception while checking and setting zookeeper ACLs", e);
241     }
242   }
243 
244   /**
245    * Set the znode perms recursively. This will do post-order recursion, so that baseZnode ACLs
246    * will be set last in case the master fails in between.
247    * @param znode
248    */
249   private void setZnodeAclsRecursive(String znode) throws KeeperException, InterruptedException {
250     List<String> children = recoverableZooKeeper.getChildren(znode, false);
251 
252     for (String child : children) {
253       setZnodeAclsRecursive(ZKUtil.joinZNode(znode, child));
254     }
255     List<ACL> acls = ZKUtil.createACL(this, znode, true);
256     LOG.info("Setting ACLs for znode:" + znode + " , acl:" + acls);
257     recoverableZooKeeper.setAcl(znode, acls, -1);
258   }
259 
260   /**
261    * Checks whether the ACLs returned from the base znode (/hbase) is set for secure setup.
262    * @param acls acls from zookeeper
263    * @return whether ACLs are set for the base znode
264    * @throws IOException
265    */
266   private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException {
267     String superUser = conf.get("hbase.superuser");
268 
269     // this assumes that current authenticated user is the same as zookeeper client user
270     // configured via JAAS
271     String hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
272 
273     if (acls.isEmpty()) {
274       return false;
275     }
276 
277     for (ACL acl : acls) {
278       int perms = acl.getPerms();
279       Id id = acl.getId();
280       // We should only set at most 3 possible ACLs for 3 Ids. One for everyone, one for superuser
281       // and one for the hbase user
282       if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
283         if (perms != Perms.READ) {
284           return false;
285         }
286       } else if (superUser != null && new Id("sasl", superUser).equals(id)) {
287         if (perms != Perms.ALL) {
288           return false;
289         }
290       } else if (new Id("sasl", hbaseUser).equals(id)) {
291         if (perms != Perms.ALL) {
292           return false;
293         }
294       } else {
295         return false;
296       }
297     }
298     return true;
299   }
300 
301   @Override
302   public String toString() {
303     return this.identifier + ", quorum=" + quorum + ", baseZNode=" + baseZNode;
304   }
305 
306   /**
307    * Adds this instance's identifier as a prefix to the passed <code>str</code>
308    * @param str String to amend.
309    * @return A new string with this instance's identifier as prefix: e.g.
310    * if passed 'hello world', the returned string could be
311    */
312   public String prefix(final String str) {
313     return this.toString() + " " + str;
314   }
315 
316   /**
317    * Set the local variable node names using the specified configuration.
318    */
319   private void setNodeNames(Configuration conf) {
320     baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
321         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
322     metaReplicaZnodes.put(0, ZKUtil.joinZNode(baseZNode,
323            conf.get("zookeeper.znode.metaserver", "meta-region-server")));
324     int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
325             HConstants.DEFAULT_META_REPLICA_NUM);
326     for (int i = 1; i < numMetaReplicas; i++) {
327       String str = ZKUtil.joinZNode(baseZNode,
328         conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + i);
329       metaReplicaZnodes.put(i, str);
330     }
331     rsZNode = ZKUtil.joinZNode(baseZNode,
332         conf.get("zookeeper.znode.rs", "rs"));
333     drainingZNode = ZKUtil.joinZNode(baseZNode,
334         conf.get("zookeeper.znode.draining.rs", "draining"));
335     masterAddressZNode = ZKUtil.joinZNode(baseZNode,
336         conf.get("zookeeper.znode.master", "master"));
337     backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode,
338         conf.get("zookeeper.znode.backup.masters", "backup-masters"));
339     clusterStateZNode = ZKUtil.joinZNode(baseZNode,
340         conf.get("zookeeper.znode.state", "running"));
341     assignmentZNode = ZKUtil.joinZNode(baseZNode,
342         conf.get("zookeeper.znode.unassigned", "region-in-transition"));
343     tableZNode = ZKUtil.joinZNode(baseZNode,
344         conf.get("zookeeper.znode.tableEnableDisable", "table"));
345     clusterIdZNode = ZKUtil.joinZNode(baseZNode,
346         conf.get("zookeeper.znode.clusterId", "hbaseid"));
347     splitLogZNode = ZKUtil.joinZNode(baseZNode,
348         conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME));
349     balancerZNode = ZKUtil.joinZNode(baseZNode,
350         conf.get("zookeeper.znode.balancer", "balancer"));
351     regionNormalizerZNode = ZKUtil.joinZNode(baseZNode,
352       conf.get("zookeeper.znode.regionNormalizer", "normalizer"));
353     switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch"));
354     tableLockZNode = ZKUtil.joinZNode(baseZNode,
355         conf.get("zookeeper.znode.tableLock", "table-lock"));
356     recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
357         conf.get("zookeeper.znode.recovering.regions", "recovering-regions"));
358     namespaceZNode = ZKUtil.joinZNode(baseZNode,
359         conf.get("zookeeper.znode.namespace", "namespace"));
360   }
361 
362   /**
363    * Is the znode of any meta replica
364    * @param node
365    * @return true or false
366    */
367   public boolean isAnyMetaReplicaZnode(String node) {
368     if (metaReplicaZnodes.values().contains(node)) {
369       return true;
370     }
371     return false;
372   }
373 
374   /**
375    * Is it the default meta replica's znode
376    * @param node
377    * @return true or false
378    */
379   public boolean isDefaultMetaReplicaZnode(String node) {
380     if (getZNodeForReplica(HRegionInfo.DEFAULT_REPLICA_ID).equals(node)) {
381       return true;
382     }
383     return false;
384   }
385 
386   /**
387    * Get the znodes corresponding to the meta replicas from ZK
388    * @return list of znodes
389    * @throws KeeperException
390    */
391   public List<String> getMetaReplicaNodes() throws KeeperException {
392     List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, baseZNode);
393     List<String> metaReplicaNodes = new ArrayList<String>(2);
394     String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
395     for (String child : childrenOfBaseNode) {
396       if (child.startsWith(pattern)) metaReplicaNodes.add(child);
397     }
398     return metaReplicaNodes;
399   }
400 
401   /**
402    * Get the znode string corresponding to a replicaId
403    * @param replicaId
404    * @return znode
405    */
406   public String getZNodeForReplica(int replicaId) {
407     String str = metaReplicaZnodes.get(replicaId);
408     // return a newly created path but don't update the cache of paths
409     // This is mostly needed for tests that attempt to create meta replicas
410     // from outside the master
411     if (str == null) {
412       str = ZKUtil.joinZNode(baseZNode,
413           conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + replicaId);
414     }
415     return str;
416   }
417 
418   /**
419    * Parse the meta replicaId from the passed znode
420    * @param znode
421    * @return replicaId
422    */
423   public int getMetaReplicaIdFromZnode(String znode) {
424     String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
425     if (znode.equals(pattern)) return HRegionInfo.DEFAULT_REPLICA_ID;
426     // the non-default replicas are of the pattern meta-region-server-<replicaId>
427     String nonDefaultPattern = pattern + "-";
428     return Integer.parseInt(znode.substring(nonDefaultPattern.length()));
429   }
430 
431   /**
432    * Register the specified listener to receive ZooKeeper events.
433    * @param listener
434    */
435   public void registerListener(ZooKeeperListener listener) {
436     listeners.add(listener);
437   }
438 
439   /**
440    * Register the specified listener to receive ZooKeeper events and add it as
441    * the first in the list of current listeners.
442    * @param listener
443    */
444   public void registerListenerFirst(ZooKeeperListener listener) {
445     listeners.add(0, listener);
446   }
447 
448   public void unregisterListener(ZooKeeperListener listener) {
449     listeners.remove(listener);
450   }
451 
452   /**
453    * Clean all existing listeners
454    */
455   public void unregisterAllListeners() {
456     listeners.clear();
457   }
458 
459   /**
460    * Get a copy of current registered listeners
461    */
462   public List<ZooKeeperListener> getListeners() {
463     return new ArrayList<ZooKeeperListener>(listeners);
464   }
465 
466   /**
467    * @return The number of currently registered listeners
468    */
469   public int getNumberOfListeners() {
470     return listeners.size();
471   }
472 
473   /**
474    * Get the connection to ZooKeeper.
475    * @return connection reference to zookeeper
476    */
477   public RecoverableZooKeeper getRecoverableZooKeeper() {
478     return recoverableZooKeeper;
479   }
480 
481   public void reconnectAfterExpiration() throws IOException, KeeperException, InterruptedException {
482     recoverableZooKeeper.reconnectAfterExpiration();
483   }
484 
485   /**
486    * Get the quorum address of this instance.
487    * @return quorum string of this zookeeper connection instance
488    */
489   public String getQuorum() {
490     return quorum;
491   }
492 
493   /**
494    * @return the base znode of this zookeeper connection instance.
495    */
496   public String getBaseZNode() {
497     return baseZNode;
498   }
499 
500   /**
501    * Method called from ZooKeeper for events and connection status.
502    * <p>
503    * Valid events are passed along to listeners.  Connection status changes
504    * are dealt with locally.
505    */
506   @Override
507   public void process(WatchedEvent event) {
508     LOG.debug(prefix("Received ZooKeeper Event, " +
509         "type=" + event.getType() + ", " +
510         "state=" + event.getState() + ", " +
511         "path=" + event.getPath()));
512 
513     switch(event.getType()) {
514 
515       // If event type is NONE, this is a connection status change
516       case None: {
517         connectionEvent(event);
518         break;
519       }
520 
521       // Otherwise pass along to the listeners
522 
523       case NodeCreated: {
524         for(ZooKeeperListener listener : listeners) {
525           listener.nodeCreated(event.getPath());
526         }
527         break;
528       }
529 
530       case NodeDeleted: {
531         for(ZooKeeperListener listener : listeners) {
532           listener.nodeDeleted(event.getPath());
533         }
534         break;
535       }
536 
537       case NodeDataChanged: {
538         for(ZooKeeperListener listener : listeners) {
539           listener.nodeDataChanged(event.getPath());
540         }
541         break;
542       }
543 
544       case NodeChildrenChanged: {
545         for(ZooKeeperListener listener : listeners) {
546           listener.nodeChildrenChanged(event.getPath());
547         }
548         break;
549       }
550     }
551   }
552 
553   // Connection management
554 
555   /**
556    * Called when there is a connection-related event via the Watcher callback.
557    * <p>
558    * If Disconnected or Expired, this should shutdown the cluster. But, since
559    * we send a KeeperException.SessionExpiredException along with the abort
560    * call, it's possible for the Abortable to catch it and try to create a new
561    * session with ZooKeeper. This is what the client does in HCM.
562    * <p>
563    * @param event
564    */
565   private void connectionEvent(WatchedEvent event) {
566     switch(event.getState()) {
567       case SyncConnected:
568         this.identifier = this.prefix + "-0x" +
569           Long.toHexString(this.recoverableZooKeeper.getSessionId());
570         // Update our identifier.  Otherwise ignore.
571         LOG.debug(this.identifier + " connected");
572         break;
573 
574       // Abort the server if Disconnected or Expired
575       case Disconnected:
576         LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
577         break;
578 
579       case Expired:
580         String msg = prefix(this.identifier + " received expired from " +
581           "ZooKeeper, aborting");
582         // TODO: One thought is to add call to ZooKeeperListener so say,
583         // ZooKeeperNodeTracker can zero out its data values.
584         if (this.abortable != null) {
585           this.abortable.abort(msg, new KeeperException.SessionExpiredException());
586         }
587         break;
588 
589       case ConnectedReadOnly:
590       case SaslAuthenticated:
591       case AuthFailed:
592         break;
593 
594       default:
595         throw new IllegalStateException("Received event is not valid: " + event.getState());
596     }
597   }
598 
599   /**
600    * Forces a synchronization of this ZooKeeper client connection.
601    * <p>
602    * Executing this method before running other methods will ensure that the
603    * subsequent operations are up-to-date and consistent as of the time that
604    * the sync is complete.
605    * <p>
606    * This is used for compareAndSwap type operations where we need to read the
607    * data of an existing node and delete or transition that node, utilizing the
608    * previously read version and data.  We want to ensure that the version read
609    * is up-to-date from when we begin the operation.
610    */
611   public void sync(String path) throws KeeperException {
612     this.recoverableZooKeeper.sync(path, null, null);
613   }
614 
615   /**
616    * Handles KeeperExceptions in client calls.
617    * <p>
618    * This may be temporary but for now this gives one place to deal with these.
619    * <p>
620    * TODO: Currently this method rethrows the exception to let the caller handle
621    * <p>
622    * @param ke
623    * @throws KeeperException
624    */
625   public void keeperException(KeeperException ke)
626   throws KeeperException {
627     LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
628     throw ke;
629   }
630 
631   /**
632    * Handles InterruptedExceptions in client calls.
633    * <p>
634    * This may be temporary but for now this gives one place to deal with these.
635    * <p>
636    * TODO: Currently, this method does nothing.
637    *       Is this ever expected to happen?  Do we abort or can we let it run?
638    *       Maybe this should be logged as WARN?  It shouldn't happen?
639    * <p>
640    * @param ie
641    */
642   public void interruptedException(InterruptedException ie) {
643     LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie);
644     // At least preserver interrupt.
645     Thread.currentThread().interrupt();
646     // no-op
647   }
648 
649   /**
650    * Close the connection to ZooKeeper.
651    *
652    * @throws InterruptedException
653    */
654   @Override
655   public void close() {
656     try {
657       recoverableZooKeeper.close();
658     } catch (InterruptedException e) {
659       Thread.currentThread().interrupt();
660     }
661   }
662 
663   public Configuration getConfiguration() {
664     return conf;
665   }
666 
667   @Override
668   public void abort(String why, Throwable e) {
669     if (this.abortable != null) this.abortable.abort(why, e);
670     else this.aborted = true;
671   }
672 
673   @Override
674   public boolean isAborted() {
675     return this.abortable == null? this.aborted: this.abortable.isAborted();
676   }
677 
678   /**
679    * @return Path to the currently active master.
680    */
681   public String getMasterAddressZNode() {
682     return this.masterAddressZNode;
683   }
684 
685   /**
686    * @return ZooKeeper znode for region normalizer state
687    */
688   public String getRegionNormalizerZNode() {
689     return regionNormalizerZNode;
690   }
691 
692   /**
693    *  @return ZK node for switch
694    * */
695   public String getSwitchZNode() {
696     return switchZNode;
697   }
698 }