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