1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.zookeeper;
20
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.classification.InterfaceAudience;
24 import org.apache.hadoop.classification.InterfaceStability;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.Abortable;
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
29 import org.apache.hadoop.hbase.util.Threads;
30 import org.apache.zookeeper.KeeperException;
31 import org.apache.zookeeper.WatchedEvent;
32 import org.apache.zookeeper.Watcher;
33 import org.apache.zookeeper.ZooDefs;
34 import org.apache.zookeeper.data.ACL;
35
36 import java.io.Closeable;
37 import java.io.IOException;
38 import java.util.ArrayList;
39 import java.util.List;
40 import java.util.concurrent.CopyOnWriteArrayList;
41 import java.util.concurrent.CountDownLatch;
42
43
44
45
46
47
48
49
50
51
52
53
54 @InterfaceAudience.Public
55 @InterfaceStability.Evolving
56 public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
57 private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
58
59
60
61 private String identifier;
62
63
64 private String quorum;
65
66
67 private RecoverableZooKeeper recoverableZooKeeper;
68
69
70 protected Abortable abortable;
71
72 private boolean aborted = false;
73
74
75 private final List<ZooKeeperListener> listeners =
76 new CopyOnWriteArrayList<ZooKeeperListener>();
77
78
79
80 public CountDownLatch saslLatch = new CountDownLatch(1);
81
82
83
84
85 public String baseZNode;
86
87 public String metaServerZNode;
88
89 public String rsZNode;
90
91 public String drainingZNode;
92
93 private String masterAddressZNode;
94
95 public String backupMasterAddressesZNode;
96
97 public String clusterStateZNode;
98
99 public String assignmentZNode;
100
101 public String tableZNode;
102
103 public String clusterIdZNode;
104
105 public String splitLogZNode;
106
107 public String balancerZNode;
108
109 public String tableLockZNode;
110
111 public String recoveringRegionsZNode;
112
113 public static String namespaceZNode = "namespace";
114
115
116
117 public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
118 new ArrayList<ACL>() { {
119 add(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
120 add(new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
121 }};
122
123 private final Configuration conf;
124
125 private final Exception constructorCaller;
126
127
128
129
130
131
132
133
134 public ZooKeeperWatcher(Configuration conf, String identifier,
135 Abortable abortable) throws ZooKeeperConnectionException, IOException {
136 this(conf, identifier, abortable, false);
137 }
138
139
140
141
142
143
144
145
146
147
148
149
150 public ZooKeeperWatcher(Configuration conf, String identifier,
151 Abortable abortable, boolean canCreateBaseZNode)
152 throws IOException, ZooKeeperConnectionException {
153 this.conf = conf;
154
155
156 try {
157 throw new Exception("ZKW CONSTRUCTOR STACK TRACE FOR DEBUGGING");
158 } catch (Exception e) {
159 this.constructorCaller = e;
160 }
161 this.quorum = ZKConfig.getZKQuorumServersString(conf);
162
163
164 this.identifier = identifier;
165 this.abortable = abortable;
166 setNodeNames(conf);
167 this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, identifier);
168 if (canCreateBaseZNode) {
169 createBaseZNodes();
170 }
171 }
172
173 private void createBaseZNodes() throws ZooKeeperConnectionException {
174 try {
175
176 ZKUtil.createWithParents(this, baseZNode);
177 ZKUtil.createAndFailSilent(this, assignmentZNode);
178 ZKUtil.createAndFailSilent(this, rsZNode);
179 ZKUtil.createAndFailSilent(this, drainingZNode);
180 ZKUtil.createAndFailSilent(this, tableZNode);
181 ZKUtil.createAndFailSilent(this, splitLogZNode);
182 ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode);
183 ZKUtil.createAndFailSilent(this, tableLockZNode);
184 ZKUtil.createAndFailSilent(this, recoveringRegionsZNode);
185 } catch (KeeperException e) {
186 throw new ZooKeeperConnectionException(
187 prefix("Unexpected KeeperException creating base node"), e);
188 }
189 }
190
191 @Override
192 public String toString() {
193 return this.identifier;
194 }
195
196
197
198
199
200
201
202 public String prefix(final String str) {
203 return this.toString() + " " + str;
204 }
205
206
207
208
209 private void setNodeNames(Configuration conf) {
210 baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
211 HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
212 metaServerZNode = ZKUtil.joinZNode(baseZNode,
213 conf.get("zookeeper.znode.metaserver", "meta-region-server"));
214 rsZNode = ZKUtil.joinZNode(baseZNode,
215 conf.get("zookeeper.znode.rs", "rs"));
216 drainingZNode = ZKUtil.joinZNode(baseZNode,
217 conf.get("zookeeper.znode.draining.rs", "draining"));
218 masterAddressZNode = ZKUtil.joinZNode(baseZNode,
219 conf.get("zookeeper.znode.master", "master"));
220 backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode,
221 conf.get("zookeeper.znode.backup.masters", "backup-masters"));
222 clusterStateZNode = ZKUtil.joinZNode(baseZNode,
223 conf.get("zookeeper.znode.state", "running"));
224 assignmentZNode = ZKUtil.joinZNode(baseZNode,
225 conf.get("zookeeper.znode.unassigned", "region-in-transition"));
226 tableZNode = ZKUtil.joinZNode(baseZNode,
227 conf.get("zookeeper.znode.tableEnableDisable", "table"));
228 clusterIdZNode = ZKUtil.joinZNode(baseZNode,
229 conf.get("zookeeper.znode.clusterId", "hbaseid"));
230 splitLogZNode = ZKUtil.joinZNode(baseZNode,
231 conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME));
232 balancerZNode = ZKUtil.joinZNode(baseZNode,
233 conf.get("zookeeper.znode.balancer", "balancer"));
234 tableLockZNode = ZKUtil.joinZNode(baseZNode,
235 conf.get("zookeeper.znode.tableLock", "table-lock"));
236 recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
237 conf.get("zookeeper.znode.recovering.regions", "recovering-regions"));
238 namespaceZNode = ZKUtil.joinZNode(baseZNode,
239 conf.get("zookeeper.znode.namespace", "namespace"));
240 }
241
242
243
244
245
246 public void registerListener(ZooKeeperListener listener) {
247 listeners.add(listener);
248 }
249
250
251
252
253
254
255 public void registerListenerFirst(ZooKeeperListener listener) {
256 listeners.add(0, listener);
257 }
258
259 public void unregisterListener(ZooKeeperListener listener) {
260 listeners.remove(listener);
261 }
262
263
264
265
266 public void unregisterAllListeners() {
267 listeners.clear();
268 }
269
270
271
272
273 public List<ZooKeeperListener> getListeners() {
274 return new ArrayList<ZooKeeperListener>(listeners);
275 }
276
277
278
279
280 public int getNumberOfListeners() {
281 return listeners.size();
282 }
283
284
285
286
287
288 public RecoverableZooKeeper getRecoverableZooKeeper() {
289 return recoverableZooKeeper;
290 }
291
292 public void reconnectAfterExpiration() throws IOException, InterruptedException {
293 recoverableZooKeeper.reconnectAfterExpiration();
294 }
295
296
297
298
299
300 public String getQuorum() {
301 return quorum;
302 }
303
304
305
306
307
308
309
310 @Override
311 public void process(WatchedEvent event) {
312 LOG.debug(prefix("Received ZooKeeper Event, " +
313 "type=" + event.getType() + ", " +
314 "state=" + event.getState() + ", " +
315 "path=" + event.getPath()));
316
317 switch(event.getType()) {
318
319
320 case None: {
321 connectionEvent(event);
322 break;
323 }
324
325
326
327 case NodeCreated: {
328 for(ZooKeeperListener listener : listeners) {
329 listener.nodeCreated(event.getPath());
330 }
331 break;
332 }
333
334 case NodeDeleted: {
335 for(ZooKeeperListener listener : listeners) {
336 listener.nodeDeleted(event.getPath());
337 }
338 break;
339 }
340
341 case NodeDataChanged: {
342 for(ZooKeeperListener listener : listeners) {
343 listener.nodeDataChanged(event.getPath());
344 }
345 break;
346 }
347
348 case NodeChildrenChanged: {
349 for(ZooKeeperListener listener : listeners) {
350 listener.nodeChildrenChanged(event.getPath());
351 }
352 break;
353 }
354 }
355 }
356
357
358
359
360
361
362
363
364
365
366
367
368
369 private void connectionEvent(WatchedEvent event) {
370 switch(event.getState()) {
371 case SyncConnected:
372
373
374 long finished = System.currentTimeMillis() +
375 this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait", 2000);
376 while (System.currentTimeMillis() < finished) {
377 Threads.sleep(1);
378 if (this.recoverableZooKeeper != null) break;
379 }
380 if (this.recoverableZooKeeper == null) {
381 LOG.error("ZK is null on connection event -- see stack trace " +
382 "for the stack trace when constructor was called on this zkw",
383 this.constructorCaller);
384 throw new NullPointerException("ZK is null");
385 }
386 this.identifier = this.identifier + "-0x" +
387 Long.toHexString(this.recoverableZooKeeper.getSessionId());
388
389 LOG.debug(this.identifier + " connected");
390 break;
391
392
393 case Disconnected:
394 LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
395 break;
396
397 case Expired:
398 String msg = prefix(this.identifier + " received expired from " +
399 "ZooKeeper, aborting");
400
401
402 if (this.abortable != null) {
403 this.abortable.abort(msg, new KeeperException.SessionExpiredException());
404 }
405 break;
406
407 case ConnectedReadOnly:
408 case SaslAuthenticated:
409 break;
410
411 default:
412 throw new IllegalStateException("Received event is not valid: " + event.getState());
413 }
414 }
415
416
417
418
419
420
421
422
423
424
425
426
427
428 public void sync(String path) {
429 this.recoverableZooKeeper.sync(path, null, null);
430 }
431
432
433
434
435
436
437
438
439
440
441
442 public void keeperException(KeeperException ke)
443 throws KeeperException {
444 LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
445 throw ke;
446 }
447
448
449
450
451
452
453
454
455
456
457
458
459 public void interruptedException(InterruptedException ie) {
460 LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie);
461
462 Thread.currentThread().interrupt();
463
464 }
465
466
467
468
469
470
471 public void close() {
472 try {
473 if (recoverableZooKeeper != null) {
474 recoverableZooKeeper.close();
475 }
476 } catch (InterruptedException e) {
477 Thread.currentThread().interrupt();
478 }
479 }
480
481 public Configuration getConfiguration() {
482 return conf;
483 }
484
485 @Override
486 public void abort(String why, Throwable e) {
487 if (this.abortable != null) this.abortable.abort(why, e);
488 else this.aborted = true;
489 }
490
491 @Override
492 public boolean isAborted() {
493 return this.abortable == null? this.aborted: this.abortable.isAborted();
494 }
495
496
497
498
499 public String getMasterAddressZNode() {
500 return this.masterAddressZNode;
501 }
502
503 }