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 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.hbase.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.zookeeper.KeeperException;
36 import org.apache.zookeeper.WatchedEvent;
37 import org.apache.zookeeper.Watcher;
38 import org.apache.zookeeper.ZooDefs;
39 import org.apache.zookeeper.data.ACL;
40
41
42
43
44
45
46
47
48
49
50
51
52 @InterfaceAudience.Private
53 public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
54 private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
55
56
57
58 private String prefix;
59 private String identifier;
60
61
62 private String quorum;
63
64
65 private RecoverableZooKeeper recoverableZooKeeper;
66
67
68 protected Abortable abortable;
69
70 private boolean aborted = false;
71
72
73 private final List<ZooKeeperListener> listeners =
74 new CopyOnWriteArrayList<ZooKeeperListener>();
75
76
77
78 public CountDownLatch saslLatch = new CountDownLatch(1);
79
80
81
82
83 public String baseZNode;
84
85 public String metaServerZNode;
86
87 public String rsZNode;
88
89 public String drainingZNode;
90
91 private String masterAddressZNode;
92
93 public String backupMasterAddressesZNode;
94
95 public String clusterStateZNode;
96
97 public String assignmentZNode;
98
99 public String tableZNode;
100
101 public String clusterIdZNode;
102
103 public String splitLogZNode;
104
105 public String balancerZNode;
106
107 public String tableLockZNode;
108
109 public String recoveringRegionsZNode;
110
111 public static String namespaceZNode = "namespace";
112
113
114 public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
115 new ArrayList<ACL>() { {
116 add(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
117 add(new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
118 }};
119
120 private final Configuration conf;
121
122 private final Exception constructorCaller;
123
124
125
126
127
128
129
130
131 public ZooKeeperWatcher(Configuration conf, String identifier,
132 Abortable abortable) throws ZooKeeperConnectionException, IOException {
133 this(conf, identifier, abortable, false);
134 }
135
136
137
138
139
140
141
142
143
144
145
146
147 public ZooKeeperWatcher(Configuration conf, String identifier,
148 Abortable abortable, boolean canCreateBaseZNode)
149 throws IOException, ZooKeeperConnectionException {
150 this.conf = conf;
151
152
153 try {
154 throw new Exception("ZKW CONSTRUCTOR STACK TRACE FOR DEBUGGING");
155 } catch (Exception e) {
156 this.constructorCaller = e;
157 }
158 this.quorum = ZKConfig.getZKQuorumServersString(conf);
159 this.prefix = identifier;
160
161
162 this.identifier = identifier + "0x0";
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
174 ZKUtil.createWithParents(this, baseZNode);
175 if (conf.getBoolean("hbase.assignment.usezk", true)) {
176 ZKUtil.createAndFailSilent(this, assignmentZNode);
177 }
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 + ", quorum=" + quorum + ", baseZNode=" + baseZNode;
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, KeeperException, InterruptedException {
293 recoverableZooKeeper.reconnectAfterExpiration();
294 }
295
296
297
298
299
300 public String getQuorum() {
301 return quorum;
302 }
303
304
305
306
307 public String getBaseZNode() {
308 return baseZNode;
309 }
310
311
312
313
314
315
316
317 @Override
318 public void process(WatchedEvent event) {
319 LOG.debug(prefix("Received ZooKeeper Event, " +
320 "type=" + event.getType() + ", " +
321 "state=" + event.getState() + ", " +
322 "path=" + event.getPath()));
323
324 switch(event.getType()) {
325
326
327 case None: {
328 connectionEvent(event);
329 break;
330 }
331
332
333
334 case NodeCreated: {
335 for(ZooKeeperListener listener : listeners) {
336 listener.nodeCreated(event.getPath());
337 }
338 break;
339 }
340
341 case NodeDeleted: {
342 for(ZooKeeperListener listener : listeners) {
343 listener.nodeDeleted(event.getPath());
344 }
345 break;
346 }
347
348 case NodeDataChanged: {
349 for(ZooKeeperListener listener : listeners) {
350 listener.nodeDataChanged(event.getPath());
351 }
352 break;
353 }
354
355 case NodeChildrenChanged: {
356 for(ZooKeeperListener listener : listeners) {
357 listener.nodeChildrenChanged(event.getPath());
358 }
359 break;
360 }
361 }
362 }
363
364
365
366
367
368
369
370
371
372
373
374
375
376 private void connectionEvent(WatchedEvent event) {
377 switch(event.getState()) {
378 case SyncConnected:
379
380
381 long finished = System.currentTimeMillis() +
382 this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait", 2000);
383 while (System.currentTimeMillis() < finished) {
384 try {
385 Thread.sleep(1);
386 } catch (InterruptedException e) {
387 LOG.warn("Interrupted while sleeping");
388 throw new RuntimeException("Interrupted while waiting for" +
389 " recoverableZooKeeper is set");
390 }
391 if (this.recoverableZooKeeper != null) break;
392 }
393
394 if (this.recoverableZooKeeper == null) {
395 LOG.error("ZK is null on connection event -- see stack trace " +
396 "for the stack trace when constructor was called on this zkw",
397 this.constructorCaller);
398 throw new NullPointerException("ZK is null");
399 }
400 this.identifier = this.prefix + "-0x" +
401 Long.toHexString(this.recoverableZooKeeper.getSessionId());
402
403 LOG.debug(this.identifier + " connected");
404 break;
405
406
407 case Disconnected:
408 LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
409 break;
410
411 case Expired:
412 String msg = prefix(this.identifier + " received expired from " +
413 "ZooKeeper, aborting");
414
415
416 if (this.abortable != null) {
417 this.abortable.abort(msg, new KeeperException.SessionExpiredException());
418 }
419 break;
420
421 case ConnectedReadOnly:
422 case SaslAuthenticated:
423 case AuthFailed:
424 break;
425
426 default:
427 throw new IllegalStateException("Received event is not valid: " + event.getState());
428 }
429 }
430
431
432
433
434
435
436
437
438
439
440
441
442
443 public void sync(String path) throws KeeperException {
444 this.recoverableZooKeeper.sync(path, null, null);
445 }
446
447
448
449
450
451
452
453
454
455
456
457 public void keeperException(KeeperException ke)
458 throws KeeperException {
459 LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
460 throw ke;
461 }
462
463
464
465
466
467
468
469
470
471
472
473
474 public void interruptedException(InterruptedException ie) {
475 LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie);
476
477 Thread.currentThread().interrupt();
478
479 }
480
481
482
483
484
485
486 public void close() {
487 try {
488 if (recoverableZooKeeper != null) {
489 recoverableZooKeeper.close();
490 }
491 } catch (InterruptedException e) {
492 Thread.currentThread().interrupt();
493 }
494 }
495
496 public Configuration getConfiguration() {
497 return conf;
498 }
499
500 @Override
501 public void abort(String why, Throwable e) {
502 if (this.abortable != null) this.abortable.abort(why, e);
503 else this.aborted = true;
504 }
505
506 @Override
507 public boolean isAborted() {
508 return this.abortable == null? this.aborted: this.abortable.isAborted();
509 }
510
511
512
513
514 public String getMasterAddressZNode() {
515 return this.masterAddressZNode;
516 }
517
518 }