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