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.List;
25 import java.util.concurrent.CopyOnWriteArrayList;
26 import java.util.concurrent.CountDownLatch;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.Abortable;
33 import org.apache.hadoop.hbase.HConstants;
34 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
35 import org.apache.hadoop.hbase.util.Threads;
36 import org.apache.zookeeper.KeeperException;
37 import org.apache.zookeeper.WatchedEvent;
38 import org.apache.zookeeper.Watcher;
39 import org.apache.zookeeper.ZooDefs;
40 import org.apache.zookeeper.data.ACL;
41
42 /**
43 * Acts as the single ZooKeeper Watcher. One instance of this is instantiated
44 * for each Master, RegionServer, and client process.
45 *
46 * <p>This is the only class that implements {@link Watcher}. Other internal
47 * classes which need to be notified of ZooKeeper events must register with
48 * the local instance of this watcher via {@link #registerListener}.
49 *
50 * <p>This class also holds and manages the connection to ZooKeeper. Code to
51 * deal with connection related events and exceptions are handled here.
52 */
53 @InterfaceAudience.Private
54 public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
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 protected Abortable abortable;
69 // Used if abortable is null
70 private boolean aborted = false;
71
72 // listeners to be notified
73 private final List<ZooKeeperListener> listeners =
74 new CopyOnWriteArrayList<ZooKeeperListener>();
75
76 // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
77 // negotiation to complete
78 public CountDownLatch saslLatch = new CountDownLatch(1);
79
80 // node names
81
82 // base znode for this cluster
83 public String baseZNode;
84 // znode containing location of server hosting meta region
85 public String metaServerZNode;
86 // znode containing ephemeral nodes of the regionservers
87 public String rsZNode;
88 // znode containing ephemeral nodes of the draining regionservers
89 public String drainingZNode;
90 // znode of currently active master
91 private String masterAddressZNode;
92 // znode of this master in backup master directory, if not the active master
93 public String backupMasterAddressesZNode;
94 // znode containing the current cluster state
95 public String clusterStateZNode;
96 // znode used for region transitioning and assignment
97 public String assignmentZNode;
98 // znode used for table disabling/enabling
99 public String tableZNode;
100 // znode containing the unique cluster ID
101 public String clusterIdZNode;
102 // znode used for log splitting work assignment
103 public String splitLogZNode;
104 // znode containing the state of the load balancer
105 public String balancerZNode;
106 // znode containing the lock for the tables
107 public String tableLockZNode;
108 // znode containing the state of recovering regions
109 public String recoveringRegionsZNode;
110 // znode containing namespace descriptors
111 public static String namespaceZNode = "namespace";
112
113
114 // Certain ZooKeeper nodes need to be world-readable
115 public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
116 new ArrayList<ACL>() { {
117 add( new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
118 add( new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
119 }};
120
121 private final Configuration conf;
122
123 private final Exception constructorCaller;
124
125 /**
126 * Instantiate a ZooKeeper connection and watcher.
127 * @param identifier string that is passed to RecoverableZookeeper to be used as
128 * identifier for this instance. Use null for default.
129 * @throws IOException
130 * @throws ZooKeeperConnectionException
131 */
132 public ZooKeeperWatcher(Configuration conf, String identifier,
133 Abortable abortable) throws ZooKeeperConnectionException, IOException {
134 this(conf, identifier, abortable, false);
135 }
136
137 /**
138 * Instantiate a ZooKeeper connection and watcher.
139 * @param conf
140 * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
141 * this instance. Use null for default.
142 * @param abortable Can be null if there is on error there is no host to abort: e.g. client
143 * context.
144 * @param canCreateBaseZNode
145 * @throws IOException
146 * @throws ZooKeeperConnectionException
147 */
148 public ZooKeeperWatcher(Configuration conf, String identifier,
149 Abortable abortable, boolean canCreateBaseZNode)
150 throws IOException, ZooKeeperConnectionException {
151 this.conf = conf;
152 // Capture a stack trace now. Will print it out later if problem so we can
153 // distingush amongst the myriad ZKWs.
154 try {
155 throw new Exception( "ZKW CONSTRUCTOR STACK TRACE FOR DEBUGGING");
156 } catch (Exception e) {
157 this.constructorCaller = e;
158 }
159 this.quorum = ZKConfig.getZKQuorumServersString(conf);
160 // Identifier will get the sessionid appended later below down when we
161 // handle the syncconnect event.
162 this.identifier = identifier;
163 this.abortable = abortable;
164 setNodeNames(conf);
165 this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, identifier);
166 if (canCreateBaseZNode) {
167 createBaseZNodes();
168 }
169 }
170
171 private void createBaseZNodes() throws ZooKeeperConnectionException {
172 try {
173 // Create all the necessary "directories" of znodes
174 ZKUtil.createWithParents( this, baseZNode);
175 ZKUtil.createAndFailSilent( this, assignmentZNode);
176 ZKUtil.createAndFailSilent( this, rsZNode);
177 ZKUtil.createAndFailSilent( this, drainingZNode);
178 ZKUtil.createAndFailSilent( this, tableZNode);
179 ZKUtil.createAndFailSilent( this, splitLogZNode);
180 ZKUtil.createAndFailSilent( this, backupMasterAddressesZNode);
181 ZKUtil.createAndFailSilent( this, tableLockZNode);
182 ZKUtil.createAndFailSilent( this, recoveringRegionsZNode);
183 } catch (KeeperException e) {
184 throw new ZooKeeperConnectionException(
185 prefix( "Unexpected KeeperException creating base node"), e);
186 }
187 }
188
189 @Override
190 public String toString() {
191 return this.identifier + ", quorum=" + quorum + ", baseZNode=" + baseZNode;
192 }
193
194 /**
195 * Adds this instance's identifier as a prefix to the passed <code>str</code>
196 * @param str String to amend.
197 * @return A new string with this instance's identifier as prefix: e.g.
198 * if passed 'hello world', the returned string could be
199 */
200 public String prefix( final String str) {
201 return this.toString() + " " + str;
202 }
203
204 /**
205 * Set the local variable node names using the specified configuration.
206 */
207 private void setNodeNames(Configuration conf) {
208 baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
209 HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
210 metaServerZNode = ZKUtil.joinZNode(baseZNode,
211 conf.get( "zookeeper.znode.metaserver", "meta-region-server"));
212 rsZNode = ZKUtil.joinZNode(baseZNode,
213 conf.get( "zookeeper.znode.rs", "rs"));
214 drainingZNode = ZKUtil.joinZNode(baseZNode,
215 conf.get( "zookeeper.znode.draining.rs", "draining"));
216 masterAddressZNode = ZKUtil.joinZNode(baseZNode,
217 conf.get( "zookeeper.znode.master", "master"));
218 backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode,
219 conf.get( "zookeeper.znode.backup.masters", "backup-masters"));
220 clusterStateZNode = ZKUtil.joinZNode(baseZNode,
221 conf.get( "zookeeper.znode.state", "running"));
222 assignmentZNode = ZKUtil.joinZNode(baseZNode,
223 conf.get( "zookeeper.znode.unassigned", "region-in-transition"));
224 tableZNode = ZKUtil.joinZNode(baseZNode,
225 conf.get( "zookeeper.znode.tableEnableDisable", "table"));
226 clusterIdZNode = ZKUtil.joinZNode(baseZNode,
227 conf.get( "zookeeper.znode.clusterId", "hbaseid"));
228 splitLogZNode = ZKUtil.joinZNode(baseZNode,
229 conf.get( "zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME));
230 balancerZNode = ZKUtil.joinZNode(baseZNode,
231 conf.get( "zookeeper.znode.balancer", "balancer"));
232 tableLockZNode = ZKUtil.joinZNode(baseZNode,
233 conf.get( "zookeeper.znode.tableLock", "table-lock"));
234 recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
235 conf.get( "zookeeper.znode.recovering.regions", "recovering-regions"));
236 namespaceZNode = ZKUtil.joinZNode(baseZNode,
237 conf.get( "zookeeper.znode.namespace", "namespace"));
238 }
239
240 /**
241 * Register the specified listener to receive ZooKeeper events.
242 * @param listener
243 */
244 public void registerListener( ZooKeeperListener listener) {
245 listeners.add(listener);
246 }
247
248 /**
249 * Register the specified listener to receive ZooKeeper events and add it as
250 * the first in the list of current listeners.
251 * @param listener
252 */
253 public void registerListenerFirst( ZooKeeperListener listener) {
254 listeners.add(0, listener);
255 }
256
257 public void unregisterListener( ZooKeeperListener listener) {
258 listeners.remove(listener);
259 }
260
261 /**
262 * Clean all existing listeners
263 */
264 public void unregisterAllListeners() {
265 listeners.clear();
266 }
267
268 /**
269 * Get a copy of current registered listeners
270 */
271 public List<ZooKeeperListener> getListeners() {
272 return new ArrayList<ZooKeeperListener>(listeners);
273 }
274
275 /**
276 * @return The number of currently registered listeners
277 */
278 public int getNumberOfListeners() {
279 return listeners.size();
280 }
281
282 /**
283 * Get the connection to ZooKeeper.
284 * @return connection reference to zookeeper
285 */
286 public RecoverableZooKeeper getRecoverableZooKeeper() {
287 return recoverableZooKeeper;
288 }
289
290 public void reconnectAfterExpiration() throws IOException, InterruptedException {
291 recoverableZooKeeper.reconnectAfterExpiration();
292 }
293
294 /**
295 * Get the quorum address of this instance.
296 * @return quorum string of this zookeeper connection instance
297 */
298 public String getQuorum() {
299 return quorum;
300 }
301
302 /**
303 * Method called from ZooKeeper for events and connection status.
304 * <p>
305 * Valid events are passed along to listeners. Connection status changes
306 * are dealt with locally.
307 */
308 @Override
309 public void process(WatchedEvent event) {
310 LOG.debug(prefix( "Received ZooKeeper Event, " +
311 "type=" + event.getType() + ", " +
312 "state=" + event.getState() + ", " +
313 "path=" + event.getPath()));
314
315 switch(event.getType()) {
316
317 // If event type is NONE, this is a connection status change
318 case None: {
319 connectionEvent(event);
320 break;
321 }
322
323 // Otherwise pass along to the listeners
324
325 case NodeCreated: {
326 for( ZooKeeperListener listener : listeners) {
327 listener.nodeCreated(event.getPath());
328 }
329 break;
330 }
331
332 case NodeDeleted: {
333 for( ZooKeeperListener listener : listeners) {
334 listener.nodeDeleted(event.getPath());
335 }
336 break;
337 }
338
339 case NodeDataChanged: {
340 for( ZooKeeperListener listener : listeners) {
341 listener.nodeDataChanged(event.getPath());
342 }
343 break;
344 }
345
346 case NodeChildrenChanged: {
347 for( ZooKeeperListener listener : listeners) {
348 listener.nodeChildrenChanged(event.getPath());
349 }
350 break;
351 }
352 }
353 }
354
355 // Connection management
356
357 /**
358 * Called when there is a connection-related event via the Watcher callback.
359 * <p>
360 * If Disconnected or Expired, this should shutdown the cluster. But, since
361 * we send a KeeperException.SessionExpiredException along with the abort
362 * call, it's possible for the Abortable to catch it and try to create a new
363 * session with ZooKeeper. This is what the client does in HCM.
364 * <p>
365 * @param event
366 */
367 private void connectionEvent(WatchedEvent event) {
368 switch(event.getState()) {
369 case SyncConnected:
370 // Now, this callback can be invoked before the this.zookeeper is set.
371 // Wait a little while.
372 long finished = System.currentTimeMillis() +
373 this.conf.getLong( "hbase.zookeeper.watcher.sync.connected.wait", 2000);
374 while (System.currentTimeMillis() < finished) {
375 Threads.sleep(1);
376 if ( this.recoverableZooKeeper != null) break;
377 }
378 if ( this.recoverableZooKeeper == null) {
379 LOG.error( "ZK is null on connection event -- see stack trace " +
380 "for the stack trace when constructor was called on this zkw",
381 this.constructorCaller);
382 throw new NullPointerException( "ZK is null");
383 }
384 this.identifier = this.identifier + "-0x" +
385 Long.toHexString( this.recoverableZooKeeper.getSessionId());
386 // Update our identifier. Otherwise ignore.
387 LOG.debug( this.identifier + " connected");
388 break;
389
390 // Abort the server if Disconnected or Expired
391 case Disconnected:
392 LOG.debug(prefix( "Received Disconnected from ZooKeeper, ignoring"));
393 break;
394
395 case Expired:
396 String msg = prefix( this.identifier + " received expired from " +
397 "ZooKeeper, aborting");
398 // TODO: One thought is to add call to ZooKeeperListener so say,
399 // ZooKeeperNodeTracker can zero out its data values.
400 if ( this.abortable != null) {
401 this.abortable.abort(msg, new KeeperException.SessionExpiredException());
402 }
403 break;
404
405 case ConnectedReadOnly:
406 case SaslAuthenticated:
407 break;
408
409 default:
410 throw new IllegalStateException( "Received event is not valid: " + event.getState());
411 }
412 }
413
414 /**
415 * Forces a synchronization of this ZooKeeper client connection.
416 * <p>
417 * Executing this method before running other methods will ensure that the
418 * subsequent operations are up-to-date and consistent as of the time that
419 * the sync is complete.
420 * <p>
421 * This is used for compareAndSwap type operations where we need to read the
422 * data of an existing node and delete or transition that node, utilizing the
423 * previously read version and data. We want to ensure that the version read
424 * is up-to-date from when we begin the operation.
425 */
426 public void sync(String path) {
427 this.recoverableZooKeeper.sync(path, null, null);
428 }
429
430 /**
431 * Handles KeeperExceptions in client calls.
432 * <p>
433 * This may be temporary but for now this gives one place to deal with these.
434 * <p>
435 * TODO: Currently this method rethrows the exception to let the caller handle
436 * <p>
437 * @param ke
438 * @throws KeeperException
439 */
440 public void keeperException(KeeperException ke)
441 throws KeeperException {
442 LOG.error(prefix( "Received unexpected KeeperException, re-throwing exception"), ke);
443 throw ke;
444 }
445
446 /**
447 * Handles InterruptedExceptions in client calls.
448 * <p>
449 * This may be temporary but for now this gives one place to deal with these.
450 * <p>
451 * TODO: Currently, this method does nothing.
452 * Is this ever expected to happen? Do we abort or can we let it run?
453 * Maybe this should be logged as WARN? It shouldn't happen?
454 * <p>
455 * @param ie
456 */
457 public void interruptedException(InterruptedException ie) {
458 LOG.debug(prefix( "Received InterruptedException, doing nothing here"), ie);
459 // At least preserver interrupt.
460 Thread.currentThread().interrupt();
461 // no-op
462 }
463
464 /**
465 * Close the connection to ZooKeeper.
466 *
467 * @throws InterruptedException
468 */
469 public void close() {
470 try {
471 if (recoverableZooKeeper != null) {
472 recoverableZooKeeper.close();
473 }
474 } catch (InterruptedException e) {
475 Thread.currentThread().interrupt();
476 }
477 }
478
479 public Configuration getConfiguration() {
480 return conf;
481 }
482
483 @Override
484 public void abort(String why, Throwable e) {
485 if ( this.abortable != null) this.abortable.abort(why, e);
486 else this.aborted = true;
487 }
488
489 @Override
490 public boolean isAborted() {
491 return this.abortable == null? this.aborted: this.abortable.isAborted();
492 }
493
494 /**
495 * @return Path to the currently active master.
496 */
497 public String getMasterAddressZNode() {
498 return this.masterAddressZNode;
499 }
500
501 }
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.List;
25 import java.util.concurrent.CopyOnWriteArrayList;
26 import java.util.concurrent.CountDownLatch;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.Abortable;
33 import org.apache.hadoop.hbase.HConstants;
34 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
35 import org.apache.hadoop.hbase.util.Threads;
36 import org.apache.zookeeper.KeeperException;
37 import org.apache.zookeeper.WatchedEvent;
38 import org.apache.zookeeper.Watcher;
39 import org.apache.zookeeper.ZooDefs;
40 import org.apache.zookeeper.data.ACL;
41
42 /**
43 * Acts as the single ZooKeeper Watcher. One instance of this is instantiated
44 * for each Master, RegionServer, and client process.
45 *
46 * <p>This is the only class that implements {@link Watcher}. Other internal
47 * classes which need to be notified of ZooKeeper events must register with
48 * the local instance of this watcher via {@link #registerListener}.
49 *
50 * <p>This class also holds and manages the connection to ZooKeeper. Code to
51 * deal with connection related events and exceptions are handled here.
52 */
53 @InterfaceAudience.Private
54 public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
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 protected Abortable abortable;
69 // Used if abortable is null
70 private boolean aborted = false;
71
72 // listeners to be notified
73 private final List<ZooKeeperListener> listeners =
74 new CopyOnWriteArrayList<ZooKeeperListener>();
75
76 // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
77 // negotiation to complete
78 public CountDownLatch saslLatch = new CountDownLatch(1);
79
80 // node names
81
82 // base znode for this cluster
83 public String baseZNode;
84 // znode containing location of server hosting meta region
85 public String metaServerZNode;
86 // znode containing ephemeral nodes of the regionservers
87 public String rsZNode;
88 // znode containing ephemeral nodes of the draining regionservers
89 public String drainingZNode;
90 // znode of currently active master
91 private String masterAddressZNode;
92 // znode of this master in backup master directory, if not the active master
93 public String backupMasterAddressesZNode;
94 // znode containing the current cluster state
95 public String clusterStateZNode;
96 // znode used for region transitioning and assignment
97 public String assignmentZNode;
98 // znode used for table disabling/enabling
99 public String tableZNode;
100 // znode containing the unique cluster ID
101 public String clusterIdZNode;
102 // znode used for log splitting work assignment
103 public String splitLogZNode;
104 // znode containing the state of the load balancer
105 public String balancerZNode;
106 // znode containing the lock for the tables
107 public String tableLockZNode;
108 // znode containing the state of recovering regions
109 public String recoveringRegionsZNode;
110 // znode containing namespace descriptors
111 public static String namespaceZNode = "namespace";
112
113
114 // Certain ZooKeeper nodes need to be world-readable
115 public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
116 new ArrayList<ACL>() { {
117 add( new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
118 add( new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
119 }};
120
121 private final Configuration conf;
122
123 private final Exception constructorCaller;
124
125 /**
126 * Instantiate a ZooKeeper connection and watcher.
127 * @param identifier string that is passed to RecoverableZookeeper to be used as
128 * identifier for this instance. Use null for default.
129 * @throws IOException
130 * @throws ZooKeeperConnectionException
131 */
132 public ZooKeeperWatcher(Configuration conf, String identifier,
133 Abortable abortable) throws ZooKeeperConnectionException, IOException {
134 this(conf, identifier, abortable, false);
135 }
136
137 /**
138 * Instantiate a ZooKeeper connection and watcher.
139 * @param conf
140 * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
141 * this instance. Use null for default.
142 * @param abortable Can be null if there is on error there is no host to abort: e.g. client
143 * context.
144 * @param canCreateBaseZNode
145 * @throws IOException
146 * @throws ZooKeeperConnectionException
147 */
148 public ZooKeeperWatcher(Configuration conf, String identifier,
149 Abortable abortable, boolean canCreateBaseZNode)
150 throws IOException, ZooKeeperConnectionException {
151 this.conf = conf;
152 // Capture a stack trace now. Will print it out later if problem so we can
153 // distingush amongst the myriad ZKWs.
154 try {
155 throw new Exception( "ZKW CONSTRUCTOR STACK TRACE FOR DEBUGGING");
156 } catch (Exception e) {
157 this.constructorCaller = e;
158 }
159 this.quorum = ZKConfig.getZKQuorumServersString(conf);
160 // Identifier will get the sessionid appended later below down when we
161 // handle the syncconnect event.
162 this.identifier = identifier;
163 this.abortable = abortable;
164 setNodeNames(conf);
165 this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, identifier);
166 if (canCreateBaseZNode) {
167 createBaseZNodes();
168 }
169 }
170
171 private void createBaseZNodes() throws ZooKeeperConnectionException {
172 try {
173 // Create all the necessary "directories" of znodes
174 ZKUtil.createWithParents( this, baseZNode);
175 ZKUtil.createAndFailSilent( this, assignmentZNode);
176 ZKUtil.createAndFailSilent( this, rsZNode);
177 ZKUtil.createAndFailSilent( this, drainingZNode);
178 ZKUtil.createAndFailSilent( this, tableZNode);
179 ZKUtil.createAndFailSilent( this, splitLogZNode);
180 ZKUtil.createAndFailSilent( this, backupMasterAddressesZNode);
181 ZKUtil.createAndFailSilent( this, tableLockZNode);
182 ZKUtil.createAndFailSilent( this, recoveringRegionsZNode);
183 } catch (KeeperException e) {
184 throw new ZooKeeperConnectionException(
185 prefix( "Unexpected KeeperException creating base node"), e);
186 }
187 }
188
189 @Override
190 public String toString() {
191 return this.identifier + ", quorum=" + quorum + ", baseZNode=" + baseZNode;
192 }
193
194 /**
195 * Adds this instance's identifier as a prefix to the passed <code>str</code>
196 * @param str String to amend.
197 * @return A new string with this instance's identifier as prefix: e.g.
198 * if passed 'hello world', the returned string could be
199 */
200 public String prefix( final String str) {
201 return this.toString() + " " + str;
202 }
203
204 /**
205 * Set the local variable node names using the specified configuration.
206 */
207 private void setNodeNames(Configuration conf) {
208 baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
209 HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
210 metaServerZNode = ZKUtil.joinZNode(baseZNode,
211 conf.get( "zookeeper.znode.metaserver", "meta-region-server"));
212 rsZNode = ZKUtil.joinZNode(baseZNode,
213 conf.get( "zookeeper.znode.rs", "rs"));
214 drainingZNode = ZKUtil.joinZNode(baseZNode,
215 conf.get( "zookeeper.znode.draining.rs", "draining"));
216 masterAddressZNode = ZKUtil.joinZNode(baseZNode,
217 conf.get( "zookeeper.znode.master", "master"));
218 backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode,
219 conf.get( "zookeeper.znode.backup.masters", "backup-masters"));
220 clusterStateZNode = ZKUtil.joinZNode(baseZNode,
221 conf.get( "zookeeper.znode.state", "running"));
222 assignmentZNode = ZKUtil.joinZNode(baseZNode,
223 conf.get( "zookeeper.znode.unassigned", "region-in-transition"));
224 tableZNode = ZKUtil.joinZNode(baseZNode,
225 conf.get( "zookeeper.znode.tableEnableDisable", "table"));
226 clusterIdZNode = ZKUtil.joinZNode(baseZNode,
227 conf.get( "zookeeper.znode.clusterId", "hbaseid"));
228 splitLogZNode = ZKUtil.joinZNode(baseZNode,
229 conf.get( "zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME));
230 balancerZNode = ZKUtil.joinZNode(baseZNode,
231 conf.get( "zookeeper.znode.balancer", "balancer"));
232 tableLockZNode = ZKUtil.joinZNode(baseZNode,
233 conf.get( "zookeeper.znode.tableLock", "table-lock"));
234 recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
235 conf.get( "zookeeper.znode.recovering.regions", "recovering-regions"));
236 namespaceZNode = ZKUtil.joinZNode(baseZNode,
237 conf.get( "zookeeper.znode.namespace", "namespace"));
238 }
239
240 /**
241 * Register the specified listener to receive ZooKeeper events.
242 * @param listener
243 */
244 public void registerListener( ZooKeeperListener listener) {
245 listeners.add(listener);
246 }
247
248 /**
249 * Register the specified listener to receive ZooKeeper events and add it as
250 * the first in the list of current listeners.
251 * @param listener
252 */
253 public void registerListenerFirst( ZooKeeperListener listener) {
254 listeners.add(0, listener);
255 }
256
257 public void unregisterListener( ZooKeeperListener listener) {
258 listeners.remove(listener);
259 }
260
261 /**
262 * Clean all existing listeners
263 */
264 public void unregisterAllListeners() {
265 listeners.clear();
266 }
267
268 /**
269 * Get a copy of current registered listeners
270 */
271 public List<ZooKeeperListener> getListeners() {
272 return new ArrayList<ZooKeeperListener>(listeners);
273 }
274
275 /**
276 * @return The number of currently registered listeners
277 */
278 public int getNumberOfListeners() {
279 return listeners.size();
280 }
281
282 /**
283 * Get the connection to ZooKeeper.
284 * @return connection reference to zookeeper
285 */
286 public RecoverableZooKeeper getRecoverableZooKeeper() {
287 return recoverableZooKeeper;
288 }
289
290 public void reconnectAfterExpiration() throws IOException, InterruptedException {
291 recoverableZooKeeper.reconnectAfterExpiration();
292 }
293
294 /**
295 * Get the quorum address of this instance.
296 * @return quorum string of this zookeeper connection instance
297 */
298 public String getQuorum() {
299 return quorum;
300 }
301
302 /**
303 * Method called from ZooKeeper for events and connection status.
304 * <p>
305 * Valid events are passed along to listeners. Connection status changes
306 * are dealt with locally.
307 */
308 @Override
309 public void process(WatchedEvent event) {
310 LOG.debug(prefix( "Received ZooKeeper Event, " +
311 "type=" + event.getType() + ", " +
312 "state=" + event.getState() + ", " +
313 "path=" + event.getPath()));
314
315 switch(event.getType()) {
316
317 // If event type is NONE, this is a connection status change
318 case None: {
319 connectionEvent(event);
320 break;
321 }
322
323 // Otherwise pass along to the listeners
324
325 case NodeCreated: {
326 for( ZooKeeperListener listener : listeners) {
327 listener.nodeCreated(event.getPath());
328 }
329 break;
330 }
331
332 case NodeDeleted: {
333 for( ZooKeeperListener listener : listeners) {
334 listener.nodeDeleted(event.getPath());
335 }
336 break;
337 }
338
339 case NodeDataChanged: {
340 for( ZooKeeperListener listener : listeners) {
341 listener.nodeDataChanged(event.getPath());
342 }
343 break;
344 }
345
346 case NodeChildrenChanged: {
347 for( ZooKeeperListener listener : listeners) {
348 listener.nodeChildrenChanged(event.getPath());
349 }
350 break;
351 }
352 }
353 }
354
355 // Connection management
356
357 /**
358 * Called when there is a connection-related event via the Watcher callback.
359 * <p>
360 * If Disconnected or Expired, this should shutdown the cluster. But, since
361 * we send a KeeperException.SessionExpiredException along with the abort
362 * call, it's possible for the Abortable to catch it and try to create a new
363 * session with ZooKeeper. This is what the client does in HCM.
364 * <p>
365 * @param event
366 */
367 private void connectionEvent(WatchedEvent event) {
368 switch(event.getState()) {
369 case SyncConnected:
370 // Now, this callback can be invoked before the this.zookeeper is set.
371 // Wait a little while.
372 long finished = System.currentTimeMillis() +
373 this.conf.getLong( "hbase.zookeeper.watcher.sync.connected.wait", 2000);
374 while (System.currentTimeMillis() < finished) {
375 Threads.sleep(1);
376 if ( this.recoverableZooKeeper != null) break;
377 }
378 if ( this.recoverableZooKeeper == null) {
379 LOG.error( "ZK is null on connection event -- see stack trace " +
380 "for the stack trace when constructor was called on this zkw",
381 this.constructorCaller);
382 throw new NullPointerException( "ZK is null");
383 }
384 this.identifier = this.identifier + "-0x" +
385 Long.toHexString( this.recoverableZooKeeper.getSessionId());
386 // Update our identifier. Otherwise ignore.
387 LOG.debug( this.identifier + " connected");
388 break;
389
390 // Abort the server if Disconnected or Expired
391 case Disconnected:
392 LOG.debug(prefix( "Received Disconnected from ZooKeeper, ignoring"));
393 break;
394
395 case Expired:
396 String msg = prefix( this.identifier + " received expired from " +
397 "ZooKeeper, aborting");
398 // TODO: One thought is to add call to ZooKeeperListener so say,
399 // ZooKeeperNodeTracker can zero out its data values.
400 if ( this.abortable != null) {
401 this.abortable.abort(msg, new KeeperException.SessionExpiredException());
402 }
403 break;
404
405 case ConnectedReadOnly:
406 case SaslAuthenticated:
407 break;
408
409 default:
410 throw new IllegalStateException( "Received event is not valid: " + event.getState());
411 }
412 }
413
414 /**
415 * Forces a synchronization of this ZooKeeper client connection.
416 * <p>
417 * Executing this method before running other methods will ensure that the
418 * subsequent operations are up-to-date and consistent as of the time that
419 * the sync is complete.
420 * <p>
421 * This is used for compareAndSwap type operations where we need to read the
422 * data of an existing node and delete or transition that node, utilizing the
423 * previously read version and data. We want to ensure that the version read
424 * is up-to-date from when we begin the operation.
425 */
426 public void sync(String path) {
427 this.recoverableZooKeeper.sync(path, null, null);
428 }
429
430 /**
431 * Handles KeeperExceptions in client calls.
432 * <p>
433 * This may be temporary but for now this gives one place to deal with these.
434 * <p>
435 * TODO: Currently this method rethrows the exception to let the caller handle
436 * <p>
437 * @param ke
438 * @throws KeeperException
439 */
440 public void keeperException(KeeperException ke)
441 throws KeeperException {
442 LOG.error(prefix( "Received unexpected KeeperException, re-throwing exception"), ke);
443 throw ke;
444 }
445
446 /**
447 * Handles InterruptedExceptions in client calls.
448 * <p>
449 * This may be temporary but for now this gives one place to deal with these.
450 * <p>
451 * TODO: Currently, this method does nothing.
452 * Is this ever expected to happen? Do we abort or can we let it run?
453 * Maybe this should be logged as WARN? It shouldn't happen?
454 * <p>
455 * @param ie
456 */
457 public void interruptedException(InterruptedException ie) {
458 LOG.debug(prefix( "Received InterruptedException, doing nothing here"), ie);
459 // At least preserver interrupt.
460 Thread.currentThread().interrupt();
461 // no-op
462 }
463
464 /**
465 * Close the connection to ZooKeeper.
466 *
467 * @throws InterruptedException
468 */
469 public void close() {
470 try {
471 if (recoverableZooKeeper != null) {
472 recoverableZooKeeper.close();
473 }
474 } catch (InterruptedException e) {
475 Thread.currentThread().interrupt();
476 }
477 }
478
479 public Configuration getConfiguration() {
480 return conf;
481 }
482
483 @Override
484 public void abort(String why, Throwable e) {
485 if ( this.abortable != null) this.abortable.abort(why, e);
486 else this.aborted = true;
487 }
488
489 @Override
490 public boolean isAborted() {
491 return this.abortable == null? this.aborted: this.abortable.isAborted();
492 }
493
494 /**
495 * @return Path to the currently active master.
496 */
497 public String getMasterAddressZNode() {
498 return this.masterAddressZNode;
499 }
500
501 }