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