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.HashSet;
24 import java.util.List;
25 import java.util.Set;
26 import java.util.concurrent.CopyOnWriteArrayList;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.Abortable;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
34 import org.apache.hadoop.hbase.util.Threads;
35 import org.apache.zookeeper.KeeperException;
36 import org.apache.zookeeper.WatchedEvent;
37 import org.apache.zookeeper.Watcher;
38 import org.apache.zookeeper.ZooKeeper;
39
40
41
42
43
44
45
46
47
48
49
50
51 public class ZooKeeperWatcher implements Watcher, Abortable {
52 private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
53
54
55
56 private String identifier;
57
58
59 private String quorum;
60
61
62 private ZooKeeper zooKeeper;
63
64
65 private Abortable abortable;
66
67
68 private final List<ZooKeeperListener> listeners =
69 new CopyOnWriteArrayList<ZooKeeperListener>();
70
71
72 private Set<String> unassignedNodes = new HashSet<String>();
73
74
75
76
77 public String baseZNode;
78
79 public String rootServerZNode;
80
81 public String rsZNode;
82
83 public String masterAddressZNode;
84
85 public String clusterStateZNode;
86
87 public String assignmentZNode;
88
89 public String tableZNode;
90
91 private final Configuration conf;
92
93 private final Exception constructorCaller;
94
95
96
97
98
99
100
101
102 public ZooKeeperWatcher(Configuration conf, String descriptor,
103 Abortable abortable)
104 throws IOException, ZooKeeperConnectionException {
105 this.conf = conf;
106
107
108 try {
109 throw new Exception("ZKW CONSTRUCTOR STACK TRACE FOR DEBUGGING");
110 } catch (Exception e) {
111 this.constructorCaller = e;
112 }
113 this.quorum = ZKConfig.getZKQuorumServersString(conf);
114
115
116 this.identifier = descriptor;
117 this.abortable = abortable;
118 setNodeNames(conf);
119 this.zooKeeper = ZKUtil.connect(conf, quorum, this, descriptor);
120 try {
121
122
123
124
125
126
127
128 long wait = conf.getLong("hbase.zookeeper.recoverable.waittime", 10000);
129 long finished = System.currentTimeMillis() + wait;
130 KeeperException ke = null;
131 do {
132 try {
133 ZKUtil.createAndFailSilent(this, baseZNode);
134 ke = null;
135 break;
136 } catch (KeeperException.ConnectionLossException e) {
137 if (LOG.isDebugEnabled() && (isFinishedRetryingRecoverable(finished))) {
138 LOG.debug("Retrying zk create for another " +
139 (finished - System.currentTimeMillis()) +
140 "ms; set 'hbase.zookeeper.recoverable.waittime' to change " +
141 "wait time); " + e.getMessage());
142 }
143 ke = e;
144 }
145 } while (isFinishedRetryingRecoverable(finished));
146
147 if (ke != null) throw new ZooKeeperConnectionException(ke);
148 ZKUtil.createAndFailSilent(this, assignmentZNode);
149 ZKUtil.createAndFailSilent(this, rsZNode);
150 ZKUtil.createAndFailSilent(this, tableZNode);
151 } catch (KeeperException e) {
152 LOG.error(prefix("Unexpected KeeperException creating base node"), e);
153 throw new IOException(e);
154 }
155 }
156
157 private boolean isFinishedRetryingRecoverable(final long finished) {
158 return System.currentTimeMillis() < finished;
159 }
160
161 @Override
162 public String toString() {
163 return this.identifier;
164 }
165
166
167
168
169
170
171
172 public String prefix(final String str) {
173 return this.toString() + " " + str;
174 }
175
176
177
178
179 private void setNodeNames(Configuration conf) {
180 baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
181 HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
182 rootServerZNode = ZKUtil.joinZNode(baseZNode,
183 conf.get("zookeeper.znode.rootserver", "root-region-server"));
184 rsZNode = ZKUtil.joinZNode(baseZNode,
185 conf.get("zookeeper.znode.rs", "rs"));
186 masterAddressZNode = ZKUtil.joinZNode(baseZNode,
187 conf.get("zookeeper.znode.master", "master"));
188 clusterStateZNode = ZKUtil.joinZNode(baseZNode,
189 conf.get("zookeeper.znode.state", "shutdown"));
190 assignmentZNode = ZKUtil.joinZNode(baseZNode,
191 conf.get("zookeeper.znode.unassigned", "unassigned"));
192 tableZNode = ZKUtil.joinZNode(baseZNode,
193 conf.get("zookeeper.znode.tableEnableDisable", "table"));
194 }
195
196
197
198
199
200 public void registerListener(ZooKeeperListener listener) {
201 listeners.add(listener);
202 }
203
204
205
206
207
208
209 public void registerListenerFirst(ZooKeeperListener listener) {
210 listeners.add(0, listener);
211 }
212
213
214
215
216
217 public ZooKeeper getZooKeeper() {
218 return zooKeeper;
219 }
220
221
222
223
224
225 public String getQuorum() {
226 return quorum;
227 }
228
229
230
231
232
233
234
235 @Override
236 public void process(WatchedEvent event) {
237 LOG.debug(prefix("Received ZooKeeper Event, " +
238 "type=" + event.getType() + ", " +
239 "state=" + event.getState() + ", " +
240 "path=" + event.getPath()));
241
242 switch(event.getType()) {
243
244
245 case None: {
246 connectionEvent(event);
247 break;
248 }
249
250
251
252 case NodeCreated: {
253 for(ZooKeeperListener listener : listeners) {
254 listener.nodeCreated(event.getPath());
255 }
256 break;
257 }
258
259 case NodeDeleted: {
260 for(ZooKeeperListener listener : listeners) {
261 listener.nodeDeleted(event.getPath());
262 }
263 break;
264 }
265
266 case NodeDataChanged: {
267 for(ZooKeeperListener listener : listeners) {
268 listener.nodeDataChanged(event.getPath());
269 }
270 break;
271 }
272
273 case NodeChildrenChanged: {
274 for(ZooKeeperListener listener : listeners) {
275 listener.nodeChildrenChanged(event.getPath());
276 }
277 break;
278 }
279 }
280 }
281
282
283
284
285
286
287
288
289
290
291
292
293
294 private void connectionEvent(WatchedEvent event) {
295 switch(event.getState()) {
296 case SyncConnected:
297
298
299 long finished = System.currentTimeMillis() +
300 this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait", 2000);
301 while (System.currentTimeMillis() < finished) {
302 Threads.sleep(1);
303 if (this.zooKeeper != null) break;
304 }
305 if (this.zooKeeper == null) {
306 LOG.error("ZK is null on connection event -- see stack trace " +
307 "for the stack trace when constructor was called on this zkw",
308 this.constructorCaller);
309 throw new NullPointerException("ZK is null");
310 }
311 this.identifier = this.identifier + "-0x" +
312 Long.toHexString(this.zooKeeper.getSessionId());
313
314 LOG.debug(this.identifier + " connected");
315 break;
316
317
318
319 case Disconnected:
320 LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
321 break;
322
323 case Expired:
324 String msg = prefix(this.identifier + " received expired from " +
325 "ZooKeeper, aborting");
326
327
328 if (this.abortable != null) this.abortable.abort(msg,
329 new KeeperException.SessionExpiredException());
330 break;
331 }
332 }
333
334
335
336
337
338
339
340
341
342
343
344
345
346 public void sync(String path) {
347 this.zooKeeper.sync(path, null, null);
348 }
349
350
351
352
353
354 public Set<String> getNodes() {
355 return unassignedNodes;
356 }
357
358
359
360
361
362
363
364
365
366
367
368 public void keeperException(KeeperException ke)
369 throws KeeperException {
370 LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
371 throw ke;
372 }
373
374
375
376
377
378
379
380
381
382
383
384
385 public void interruptedException(InterruptedException ie) {
386 LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie);
387
388 Thread.currentThread().interrupt();
389
390 }
391
392
393
394
395
396 public void close() {
397 try {
398 if (zooKeeper != null) {
399 zooKeeper.close();
400
401 }
402 } catch (InterruptedException e) {
403 }
404 }
405
406 @Override
407 public void abort(String why, Throwable e) {
408 this.abortable.abort(why, e);
409 }
410 }