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