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.lang.management.ManagementFactory;
24 import java.util.ArrayList;
25 import java.util.List;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.util.Bytes;
30 import org.apache.hadoop.hbase.util.RetryCounter;
31 import org.apache.hadoop.hbase.util.RetryCounterFactory;
32 import org.apache.zookeeper.AsyncCallback;
33 import org.apache.zookeeper.CreateMode;
34 import org.apache.zookeeper.KeeperException;
35 import org.apache.zookeeper.Watcher;
36 import org.apache.zookeeper.ZooKeeper;
37 import org.apache.zookeeper.ZooKeeper.States;
38 import org.apache.zookeeper.data.ACL;
39 import org.apache.zookeeper.data.Stat;
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64 public class RecoverableZooKeeper {
65 private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
66
67 private ZooKeeper zk;
68 private final RetryCounterFactory retryCounterFactory;
69
70 private final String identifier;
71 private final byte[] id;
72 private int retryIntervalMillis;
73
74 private static final int ID_OFFSET = Bytes.SIZEOF_INT;
75
76 private static final byte MAGIC =(byte) 0XFF;
77 private static final int MAGIC_OFFSET = Bytes.SIZEOF_BYTE;
78
79 public RecoverableZooKeeper(String quorumServers, int seesionTimeout,
80 Watcher watcher, int maxRetries, int retryIntervalMillis)
81 throws IOException {
82 this.zk = new ZooKeeper(quorumServers, seesionTimeout, watcher);
83 this.retryCounterFactory =
84 new RetryCounterFactory(maxRetries, retryIntervalMillis);
85 this.retryIntervalMillis = retryIntervalMillis;
86
87
88 this.identifier = ManagementFactory.getRuntimeMXBean().getName();
89 LOG.info("The identifier of this process is " + identifier);
90 this.id = Bytes.toBytes(identifier);
91 }
92
93
94
95
96
97
98
99
100
101 public void delete(String path, int version)
102 throws InterruptedException, KeeperException {
103 RetryCounter retryCounter = retryCounterFactory.create();
104 boolean isRetry = false;
105 while (true) {
106 try {
107 zk.delete(path, version);
108 return;
109 } catch (KeeperException e) {
110 switch (e.code()) {
111 case NONODE:
112 if (isRetry) {
113 LOG.info("Node " + path + " already deleted. Assuming that a " +
114 "previous attempt succeeded.");
115 return;
116 }
117 LOG.warn("Node " + path + " already deleted, and this is not a " +
118 "retry");
119 throw e;
120
121 case CONNECTIONLOSS:
122 case OPERATIONTIMEOUT:
123 LOG.warn("Possibly transient ZooKeeper exception: " + e);
124 if (!retryCounter.shouldRetry()) {
125 LOG.error("ZooKeeper delete failed after "
126 + retryCounter.getMaxRetries() + " retries");
127 throw e;
128 }
129 break;
130
131 default:
132 throw e;
133 }
134 }
135 retryCounter.sleepUntilNextRetry();
136 retryCounter.useRetry();
137 isRetry = true;
138 }
139 }
140
141
142
143
144
145
146
147
148
149 public Stat exists(String path, Watcher watcher)
150 throws KeeperException, InterruptedException {
151 RetryCounter retryCounter = retryCounterFactory.create();
152 while (true) {
153 try {
154 return zk.exists(path, watcher);
155 } catch (KeeperException e) {
156 switch (e.code()) {
157 case CONNECTIONLOSS:
158 case OPERATIONTIMEOUT:
159 LOG.warn("Possibly transient ZooKeeper exception: " + e);
160 if (!retryCounter.shouldRetry()) {
161 LOG.error("ZooKeeper exists failed after "
162 + retryCounter.getMaxRetries() + " retries");
163 throw e;
164 }
165 break;
166
167 default:
168 throw e;
169 }
170 }
171 retryCounter.sleepUntilNextRetry();
172 retryCounter.useRetry();
173 }
174 }
175
176
177
178
179
180
181
182
183
184 public Stat exists(String path, boolean watch)
185 throws KeeperException, InterruptedException {
186 RetryCounter retryCounter = retryCounterFactory.create();
187 while (true) {
188 try {
189 return zk.exists(path, watch);
190 } catch (KeeperException e) {
191 switch (e.code()) {
192 case CONNECTIONLOSS:
193 case OPERATIONTIMEOUT:
194 LOG.warn("Possibly transient ZooKeeper exception: " + e);
195 if (!retryCounter.shouldRetry()) {
196 LOG.error("ZooKeeper exists failed after "
197 + retryCounter.getMaxRetries() + " retries");
198 throw e;
199 }
200 break;
201
202 default:
203 throw e;
204 }
205 }
206 retryCounter.sleepUntilNextRetry();
207 retryCounter.useRetry();
208 }
209 }
210
211
212
213
214
215
216
217
218
219 public List<String> getChildren(String path, Watcher watcher)
220 throws KeeperException, InterruptedException {
221 RetryCounter retryCounter = retryCounterFactory.create();
222 while (true) {
223 try {
224 return zk.getChildren(path, watcher);
225 } catch (KeeperException e) {
226 switch (e.code()) {
227 case CONNECTIONLOSS:
228 case OPERATIONTIMEOUT:
229 LOG.warn("Possibly transient ZooKeeper exception: " + e);
230 if (!retryCounter.shouldRetry()) {
231 LOG.error("ZooKeeper getChildren failed after "
232 + retryCounter.getMaxRetries() + " retries");
233 throw e;
234 }
235 break;
236
237 default:
238 throw e;
239 }
240 }
241 retryCounter.sleepUntilNextRetry();
242 retryCounter.useRetry();
243 }
244 }
245
246
247
248
249
250
251
252
253
254 public List<String> getChildren(String path, boolean watch)
255 throws KeeperException, InterruptedException {
256 RetryCounter retryCounter = retryCounterFactory.create();
257 while (true) {
258 try {
259 return zk.getChildren(path, watch);
260 } catch (KeeperException e) {
261 switch (e.code()) {
262 case CONNECTIONLOSS:
263 case OPERATIONTIMEOUT:
264 LOG.warn("Possibly transient ZooKeeper exception: " + e);
265 if (!retryCounter.shouldRetry()) {
266 LOG.error("ZooKeeper getChildren failed after "
267 + retryCounter.getMaxRetries() + " retries");
268 throw e;
269 }
270 break;
271
272 default:
273 throw e;
274 }
275 }
276 retryCounter.sleepUntilNextRetry();
277 retryCounter.useRetry();
278 }
279 }
280
281
282
283
284
285
286
287
288
289
290 public byte[] getData(String path, Watcher watcher, Stat stat)
291 throws KeeperException, InterruptedException {
292 RetryCounter retryCounter = retryCounterFactory.create();
293 while (true) {
294 try {
295 byte[] revData = zk.getData(path, watcher, stat);
296 return this.removeMetaData(revData);
297 } catch (KeeperException e) {
298 switch (e.code()) {
299 case CONNECTIONLOSS:
300 case OPERATIONTIMEOUT:
301 LOG.warn("Possibly transient ZooKeeper exception: " + e);
302 if (!retryCounter.shouldRetry()) {
303 LOG.error("ZooKeeper getData failed after "
304 + retryCounter.getMaxRetries() + " retries");
305 throw e;
306 }
307 break;
308
309 default:
310 throw e;
311 }
312 }
313 retryCounter.sleepUntilNextRetry();
314 retryCounter.useRetry();
315 }
316 }
317
318
319
320
321
322
323
324
325
326
327 public byte[] getData(String path, boolean watch, Stat stat)
328 throws KeeperException, InterruptedException {
329 RetryCounter retryCounter = retryCounterFactory.create();
330 while (true) {
331 try {
332 byte[] revData = zk.getData(path, watch, stat);
333 return this.removeMetaData(revData);
334 } catch (KeeperException e) {
335 switch (e.code()) {
336 case CONNECTIONLOSS:
337 case OPERATIONTIMEOUT:
338 LOG.warn("Possibly transient ZooKeeper exception: " + e);
339 if (!retryCounter.shouldRetry()) {
340 LOG.error("ZooKeeper getData failed after "
341 + retryCounter.getMaxRetries() + " retries");
342 throw e;
343 }
344 break;
345
346 default:
347 throw e;
348 }
349 }
350 retryCounter.sleepUntilNextRetry();
351 retryCounter.useRetry();
352 }
353 }
354
355
356
357
358
359
360
361
362
363
364
365
366 public Stat setData(String path, byte[] data, int version)
367 throws KeeperException, InterruptedException {
368 RetryCounter retryCounter = retryCounterFactory.create();
369 byte[] newData = appendMetaData(data);
370 while (true) {
371 try {
372 return zk.setData(path, newData, version);
373 } catch (KeeperException e) {
374 switch (e.code()) {
375 case CONNECTIONLOSS:
376 case OPERATIONTIMEOUT:
377 LOG.warn("Possibly transient ZooKeeper exception: " + e);
378 if (!retryCounter.shouldRetry()) {
379 LOG.error("ZooKeeper setData failed after "
380 + retryCounter.getMaxRetries() + " retries");
381 throw e;
382 }
383 break;
384 case BADVERSION:
385
386 try{
387 Stat stat = new Stat();
388 byte[] revData = zk.getData(path, false, stat);
389 int idLength = Bytes.toInt(revData, ID_OFFSET);
390 int dataLength = revData.length-ID_OFFSET-idLength;
391 int dataOffset = ID_OFFSET+idLength;
392
393 if(Bytes.compareTo(revData, ID_OFFSET, id.length,
394 revData, dataOffset, dataLength) == 0) {
395
396 return stat;
397 }
398 } catch(KeeperException keeperException){
399
400 throw keeperException;
401 }
402
403
404 default:
405 throw e;
406 }
407 }
408 retryCounter.sleepUntilNextRetry();
409 retryCounter.useRetry();
410 }
411 }
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434 public String create(String path, byte[] data, List<ACL> acl,
435 CreateMode createMode)
436 throws KeeperException, InterruptedException {
437 byte[] newData = appendMetaData(data);
438 switch (createMode) {
439 case EPHEMERAL:
440 case PERSISTENT:
441 return createNonSequential(path, newData, acl, createMode);
442
443 case EPHEMERAL_SEQUENTIAL:
444 case PERSISTENT_SEQUENTIAL:
445 return createSequential(path, newData, acl, createMode);
446
447 default:
448 throw new IllegalArgumentException("Unrecognized CreateMode: " +
449 createMode);
450 }
451 }
452
453 private String createNonSequential(String path, byte[] data, List<ACL> acl,
454 CreateMode createMode) throws KeeperException, InterruptedException {
455 RetryCounter retryCounter = retryCounterFactory.create();
456 boolean isRetry = false;
457 while (true) {
458 try {
459 return zk.create(path, data, acl, createMode);
460 } catch (KeeperException e) {
461 switch (e.code()) {
462 case NODEEXISTS:
463 if (isRetry) {
464
465
466
467 byte[] currentData = zk.getData(path, false, null);
468 if (currentData != null &&
469 Bytes.compareTo(currentData, data) == 0) {
470
471 return path;
472 }
473 LOG.error("Node " + path + " already exists with " +
474 Bytes.toStringBinary(currentData) + ", could not write " +
475 Bytes.toStringBinary(data));
476 throw e;
477 }
478 LOG.error("Node " + path + " already exists and this is not a " +
479 "retry");
480 throw e;
481
482 case CONNECTIONLOSS:
483 case OPERATIONTIMEOUT:
484 LOG.warn("Possibly transient ZooKeeper exception: " + e);
485 if (!retryCounter.shouldRetry()) {
486 LOG.error("ZooKeeper create failed after "
487 + retryCounter.getMaxRetries() + " retries");
488 throw e;
489 }
490 break;
491
492 default:
493 throw e;
494 }
495 }
496 retryCounter.sleepUntilNextRetry();
497 retryCounter.useRetry();
498 isRetry = true;
499 }
500 }
501
502 private String createSequential(String path, byte[] data,
503 List<ACL> acl, CreateMode createMode)
504 throws KeeperException, InterruptedException {
505 RetryCounter retryCounter = retryCounterFactory.create();
506 boolean first = true;
507 String newPath = path+this.identifier;
508 while (true) {
509 try {
510 if (!first) {
511
512 String previousResult = findPreviousSequentialNode(newPath);
513 if (previousResult != null) {
514 return previousResult;
515 }
516 }
517 first = false;
518 return zk.create(newPath, data, acl, createMode);
519 } catch (KeeperException e) {
520 switch (e.code()) {
521 case CONNECTIONLOSS:
522 case OPERATIONTIMEOUT:
523 LOG.warn("Possibly transient ZooKeeper exception: " + e);
524 if (!retryCounter.shouldRetry()) {
525 LOG.error("ZooKeeper create failed after "
526 + retryCounter.getMaxRetries() + " retries");
527 throw e;
528 }
529 break;
530
531 default:
532 throw e;
533 }
534 }
535 retryCounter.sleepUntilNextRetry();
536 retryCounter.useRetry();
537 }
538 }
539
540 private String findPreviousSequentialNode(String path)
541 throws KeeperException, InterruptedException {
542 int lastSlashIdx = path.lastIndexOf('/');
543 assert(lastSlashIdx != -1);
544 String parent = path.substring(0, lastSlashIdx);
545 String nodePrefix = path.substring(lastSlashIdx+1);
546
547 List<String> nodes = zk.getChildren(parent, false);
548 List<String> matching = filterByPrefix(nodes, nodePrefix);
549 for (String node : matching) {
550 String nodePath = parent + "/" + node;
551 Stat stat = zk.exists(nodePath, false);
552 if (stat != null) {
553 return nodePath;
554 }
555 }
556 return null;
557 }
558
559 public byte[] removeMetaData(byte[] data) {
560 if(data == null || data.length == 0) {
561 return data;
562 }
563
564 byte magic = data[0];
565 if(magic != MAGIC) {
566 return data;
567 }
568
569 int idLength = Bytes.toInt(data, MAGIC_OFFSET);
570 int dataLength = data.length-MAGIC_OFFSET-ID_OFFSET-idLength;
571 int dataOffset = MAGIC_OFFSET+ID_OFFSET+idLength;
572
573 byte[] newData = new byte[dataLength];
574 System.arraycopy(data, dataOffset, newData, 0, dataLength);
575
576 return newData;
577
578 }
579
580 private byte[] appendMetaData(byte[] data) {
581 if(data == null || data.length == 0){
582 return data;
583 }
584
585 byte[] newData = new byte[MAGIC_OFFSET+ID_OFFSET+id.length+data.length];
586 int pos = 0;
587 pos = Bytes.putByte(newData, pos, MAGIC);
588 pos = Bytes.putInt(newData, pos, id.length);
589 pos = Bytes.putBytes(newData, pos, id, 0, id.length);
590 pos = Bytes.putBytes(newData, pos, data, 0, data.length);
591
592 return newData;
593 }
594
595 public long getSessionId() {
596 return zk.getSessionId();
597 }
598
599 public void close() throws InterruptedException {
600 zk.close();
601 }
602
603 public States getState() {
604 return zk.getState();
605 }
606
607 public ZooKeeper getZooKeeper() {
608 return zk;
609 }
610
611 public byte[] getSessionPasswd() {
612 return zk.getSessionPasswd();
613 }
614
615 public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) {
616 this.zk.sync(path, null, null);
617 }
618
619
620
621
622
623
624
625
626
627
628 private static List<String> filterByPrefix(List<String> nodes,
629 String... prefixes) {
630 List<String> lockChildren = new ArrayList<String>();
631 for (String child : nodes){
632 for (String prefix : prefixes){
633 if (child.startsWith(prefix)){
634 lockChildren.add(child);
635 break;
636 }
637 }
638 }
639 return lockChildren;
640 }
641 }