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