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