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.util.ArrayList;
24 import java.util.LinkedList;
25 import java.util.List;
26 import java.util.Random;
27 import java.util.concurrent.TimeUnit;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.hbase.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.RetryCounter.BackoffPolicy;
35 import org.apache.hadoop.hbase.util.RetryCounter.RetryConfig;
36 import org.apache.hadoop.hbase.util.RetryCounterFactory;
37 import org.apache.htrace.Trace;
38 import org.apache.htrace.TraceScope;
39 import org.apache.zookeeper.AsyncCallback;
40 import org.apache.zookeeper.CreateMode;
41 import org.apache.zookeeper.KeeperException;
42 import org.apache.zookeeper.Op;
43 import org.apache.zookeeper.OpResult;
44 import org.apache.zookeeper.Watcher;
45 import org.apache.zookeeper.ZooDefs;
46 import org.apache.zookeeper.ZooKeeper;
47 import org.apache.zookeeper.ZooKeeper.States;
48 import org.apache.zookeeper.data.ACL;
49 import org.apache.zookeeper.data.Stat;
50 import org.apache.zookeeper.proto.CreateRequest;
51 import org.apache.zookeeper.proto.SetDataRequest;
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76 @InterfaceAudience.Private
77 public class RecoverableZooKeeper {
78 private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
79
80
81 private ZooKeeper zk;
82 private final RetryCounterFactory retryCounterFactory;
83
84 private final String identifier;
85 private final byte[] id;
86 private Watcher watcher;
87 private int sessionTimeout;
88 private String quorumServers;
89 private final Random salter;
90 private final RetryCounter authFailedRetryCounter;
91 private int maxMultiSize;
92
93
94
95
96
97
98
99
100
101 private static final byte MAGIC =(byte) 0XFF;
102 private static final int MAGIC_SIZE = Bytes.SIZEOF_BYTE;
103 private static final int ID_LENGTH_OFFSET = MAGIC_SIZE;
104 private static final int ID_LENGTH_SIZE = Bytes.SIZEOF_INT;
105
106 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
107 justification="None. Its always been this way.")
108 public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
109 Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier,
110 int authFailedRetries, int authFailedPause, int maxMultiSize) throws IOException {
111
112 this.retryCounterFactory =
113 new RetryCounterFactory(maxRetries+1, retryIntervalMillis, maxSleepTime);
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 this.maxMultiSize = maxMultiSize;
128
129 try {
130 checkZk();
131 } catch (Exception x) {
132
133 }
134
135 salter = new Random();
136
137 RetryConfig authFailedRetryConfig = new RetryConfig(
138 authFailedRetries + 1,
139 authFailedPause,
140 authFailedPause,
141 TimeUnit.MILLISECONDS,
142 new BackoffPolicy());
143 this.authFailedRetryCounter = new RetryCounter(authFailedRetryConfig);
144 }
145
146
147
148
149
150
151
152
153 public int getMaxMultiSizeLimit() {
154 return maxMultiSize;
155 }
156
157
158
159
160
161
162
163 protected synchronized ZooKeeper checkZk() throws KeeperException {
164 if (this.zk == null) {
165 this.zk = createNewZooKeeper();
166 }
167 return zk;
168 }
169
170
171
172
173
174 ZooKeeper createNewZooKeeper() throws KeeperException {
175 try {
176 return new ZooKeeper(quorumServers, sessionTimeout, watcher);
177 } catch (IOException ex) {
178 LOG.warn("Unable to create ZooKeeper Connection", ex);
179 throw new KeeperException.OperationTimeoutException();
180 }
181 }
182
183 public synchronized void reconnectAfterAuthFailure() throws InterruptedException,
184 KeeperException {
185 if (zk != null) {
186 LOG.info("Closing ZooKeeper connection which saw AUTH_FAILED, session" +
187 " was: 0x"+Long.toHexString(zk.getSessionId()));
188 zk.close();
189
190 zk = null;
191
192 if (!authFailedRetryCounter.shouldRetry()) {
193 throw new RuntimeException("Exceeded the configured retries for handling ZooKeeper"
194 + " AUTH_FAILED exceptions (" + authFailedRetryCounter.getMaxAttempts() + ")");
195 }
196
197 if (LOG.isTraceEnabled()) {
198 LOG.trace("Sleeping " + authFailedRetryCounter.getBackoffTime()
199 + "ms before re-creating ZooKeeper object after AUTH_FAILED state ("
200 + authFailedRetryCounter.getAttemptTimes() + "/"
201 + authFailedRetryCounter.getMaxAttempts() + ")");
202 }
203 authFailedRetryCounter.sleepUntilNextRetry();
204 }
205 checkZk();
206 LOG.info("Recreated a ZooKeeper, session" +
207 " is: 0x"+Long.toHexString(zk.getSessionId()));
208 }
209
210 public synchronized void reconnectAfterExpiration()
211 throws IOException, KeeperException, InterruptedException {
212 if (zk != null) {
213 LOG.info("Closing dead ZooKeeper connection, session" +
214 " was: 0x"+Long.toHexString(zk.getSessionId()));
215 zk.close();
216
217 zk = null;
218 }
219 checkZk();
220 LOG.info("Recreated a ZooKeeper, session" +
221 " is: 0x"+Long.toHexString(zk.getSessionId()));
222 }
223
224
225
226
227
228
229 public void delete(String path, int version)
230 throws InterruptedException, KeeperException {
231 TraceScope traceScope = null;
232 try {
233 traceScope = Trace.startSpan("RecoverableZookeeper.delete");
234 RetryCounter retryCounter = retryCounterFactory.create();
235 boolean isRetry = false;
236 while (true) {
237 try {
238 checkZk().delete(path, version);
239 return;
240 } catch (KeeperException e) {
241 switch (e.code()) {
242 case NONODE:
243 if (isRetry) {
244 LOG.debug("Node " + path + " already deleted. Assuming a " +
245 "previous attempt succeeded.");
246 return;
247 }
248 LOG.debug("Node " + path + " already deleted, retry=" + isRetry);
249 throw e;
250
251 case CONNECTIONLOSS:
252 case OPERATIONTIMEOUT:
253 retryOrThrow(retryCounter, e, "delete");
254 break;
255 case AUTHFAILED:
256 reconnectAfterAuthFailure();
257 retryOrThrow(retryCounter, e, "delete");
258 break;
259
260 default:
261 throw e;
262 }
263 }
264 retryCounter.sleepUntilNextRetry();
265 isRetry = true;
266 }
267 } finally {
268 if (traceScope != null) traceScope.close();
269 }
270 }
271
272
273
274
275
276 public Stat exists(String path, Watcher watcher)
277 throws KeeperException, InterruptedException {
278 TraceScope traceScope = null;
279 try {
280 traceScope = Trace.startSpan("RecoverableZookeeper.exists");
281 RetryCounter retryCounter = retryCounterFactory.create();
282 while (true) {
283 try {
284 return checkZk().exists(path, watcher);
285 } catch (KeeperException e) {
286 switch (e.code()) {
287 case CONNECTIONLOSS:
288 case OPERATIONTIMEOUT:
289 retryOrThrow(retryCounter, e, "exists");
290 break;
291 case AUTHFAILED:
292 reconnectAfterAuthFailure();
293 retryOrThrow(retryCounter, e, "exists");
294 break;
295
296 default:
297 throw e;
298 }
299 }
300 retryCounter.sleepUntilNextRetry();
301 }
302 } finally {
303 if (traceScope != null) traceScope.close();
304 }
305 }
306
307
308
309
310
311 public Stat exists(String path, boolean watch)
312 throws KeeperException, InterruptedException {
313 TraceScope traceScope = null;
314 try {
315 traceScope = Trace.startSpan("RecoverableZookeeper.exists");
316 RetryCounter retryCounter = retryCounterFactory.create();
317 while (true) {
318 try {
319 return checkZk().exists(path, watch);
320 } catch (KeeperException e) {
321 switch (e.code()) {
322 case CONNECTIONLOSS:
323 case OPERATIONTIMEOUT:
324 retryOrThrow(retryCounter, e, "exists");
325 break;
326 case AUTHFAILED:
327 reconnectAfterAuthFailure();
328 retryOrThrow(retryCounter, e, "exists");
329 break;
330
331 default:
332 throw e;
333 }
334 }
335 retryCounter.sleepUntilNextRetry();
336 }
337 } finally {
338 if (traceScope != null) traceScope.close();
339 }
340 }
341
342 private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
343 String opName) throws KeeperException {
344 LOG.debug("Possibly transient ZooKeeper, quorum=" + quorumServers + ", exception=" + e, e);
345 if (!retryCounter.shouldRetry()) {
346 LOG.error("ZooKeeper " + opName + " failed after "
347 + retryCounter.getMaxAttempts() + " attempts");
348 throw e;
349 }
350 }
351
352
353
354
355
356 public List<String> getChildren(String path, Watcher watcher)
357 throws KeeperException, InterruptedException {
358 TraceScope traceScope = null;
359 try {
360 traceScope = Trace.startSpan("RecoverableZookeeper.getChildren");
361 RetryCounter retryCounter = retryCounterFactory.create();
362 while (true) {
363 try {
364 return checkZk().getChildren(path, watcher);
365 } catch (KeeperException e) {
366 switch (e.code()) {
367 case CONNECTIONLOSS:
368 case OPERATIONTIMEOUT:
369 retryOrThrow(retryCounter, e, "getChildren");
370 break;
371 case AUTHFAILED:
372 reconnectAfterAuthFailure();
373 retryOrThrow(retryCounter, e, "getChildren");
374 break;
375
376 default:
377 throw e;
378 }
379 }
380 retryCounter.sleepUntilNextRetry();
381 }
382 } finally {
383 if (traceScope != null) traceScope.close();
384 }
385 }
386
387
388
389
390
391 public List<String> getChildren(String path, boolean watch)
392 throws KeeperException, InterruptedException {
393 TraceScope traceScope = null;
394 try {
395 traceScope = Trace.startSpan("RecoverableZookeeper.getChildren");
396 RetryCounter retryCounter = retryCounterFactory.create();
397 while (true) {
398 try {
399 return checkZk().getChildren(path, watch);
400 } catch (KeeperException e) {
401 switch (e.code()) {
402 case CONNECTIONLOSS:
403 case OPERATIONTIMEOUT:
404 retryOrThrow(retryCounter, e, "getChildren");
405 break;
406 case AUTHFAILED:
407 reconnectAfterAuthFailure();
408 retryOrThrow(retryCounter, e, "getChildren");
409 break;
410
411 default:
412 throw e;
413 }
414 }
415 retryCounter.sleepUntilNextRetry();
416 }
417 } finally {
418 if (traceScope != null) traceScope.close();
419 }
420 }
421
422
423
424
425
426 public byte[] getData(String path, Watcher watcher, Stat stat)
427 throws KeeperException, InterruptedException {
428 TraceScope traceScope = null;
429 try {
430 traceScope = Trace.startSpan("RecoverableZookeeper.getData");
431 RetryCounter retryCounter = retryCounterFactory.create();
432 while (true) {
433 try {
434 byte[] revData = checkZk().getData(path, watcher, stat);
435 return this.removeMetaData(revData);
436 } catch (KeeperException e) {
437 switch (e.code()) {
438 case CONNECTIONLOSS:
439 case OPERATIONTIMEOUT:
440 retryOrThrow(retryCounter, e, "getData");
441 break;
442 case AUTHFAILED:
443 reconnectAfterAuthFailure();
444 retryOrThrow(retryCounter, e, "getData");
445 break;
446
447 default:
448 throw e;
449 }
450 }
451 retryCounter.sleepUntilNextRetry();
452 }
453 } finally {
454 if (traceScope != null) traceScope.close();
455 }
456 }
457
458
459
460
461
462 public byte[] getData(String path, boolean watch, Stat stat)
463 throws KeeperException, InterruptedException {
464 TraceScope traceScope = null;
465 try {
466 traceScope = Trace.startSpan("RecoverableZookeeper.getData");
467 RetryCounter retryCounter = retryCounterFactory.create();
468 while (true) {
469 try {
470 byte[] revData = checkZk().getData(path, watch, stat);
471 return this.removeMetaData(revData);
472 } catch (KeeperException e) {
473 switch (e.code()) {
474 case CONNECTIONLOSS:
475 case OPERATIONTIMEOUT:
476 retryOrThrow(retryCounter, e, "getData");
477 break;
478 case AUTHFAILED:
479 reconnectAfterAuthFailure();
480 retryOrThrow(retryCounter, e, "getData");
481 break;
482
483 default:
484 throw e;
485 }
486 }
487 retryCounter.sleepUntilNextRetry();
488 }
489 } finally {
490 if (traceScope != null) traceScope.close();
491 }
492 }
493
494
495
496
497
498
499
500 public Stat setData(String path, byte[] data, int version)
501 throws KeeperException, InterruptedException {
502 TraceScope traceScope = null;
503 try {
504 traceScope = Trace.startSpan("RecoverableZookeeper.setData");
505 RetryCounter retryCounter = retryCounterFactory.create();
506 byte[] newData = appendMetaData(data);
507 boolean isRetry = false;
508 while (true) {
509 try {
510 return checkZk().setData(path, newData, version);
511 } catch (KeeperException e) {
512 switch (e.code()) {
513 case CONNECTIONLOSS:
514 case OPERATIONTIMEOUT:
515 retryOrThrow(retryCounter, e, "setData");
516 break;
517 case AUTHFAILED:
518 reconnectAfterAuthFailure();
519 retryOrThrow(retryCounter, e, "setData");
520 break;
521 case BADVERSION:
522 if (isRetry) {
523
524 try{
525 Stat stat = new Stat();
526 byte[] revData = checkZk().getData(path, false, stat);
527 if(Bytes.compareTo(revData, newData) == 0) {
528
529 return stat;
530 }
531 } catch(KeeperException keeperException){
532
533 throw keeperException;
534 }
535 }
536
537 default:
538 throw e;
539 }
540 }
541 retryCounter.sleepUntilNextRetry();
542 isRetry = true;
543 }
544 } finally {
545 if (traceScope != null) traceScope.close();
546 }
547 }
548
549
550
551
552
553 public List<ACL> getAcl(String path, Stat stat)
554 throws KeeperException, InterruptedException {
555 TraceScope traceScope = null;
556 try {
557 traceScope = Trace.startSpan("RecoverableZookeeper.getAcl");
558 RetryCounter retryCounter = retryCounterFactory.create();
559 while (true) {
560 try {
561 return checkZk().getACL(path, stat);
562 } catch (KeeperException e) {
563 switch (e.code()) {
564 case CONNECTIONLOSS:
565 case OPERATIONTIMEOUT:
566 retryOrThrow(retryCounter, e, "getAcl");
567 break;
568 case AUTHFAILED:
569 reconnectAfterAuthFailure();
570 retryOrThrow(retryCounter, e, "getAcl");
571 break;
572
573 default:
574 throw e;
575 }
576 }
577 retryCounter.sleepUntilNextRetry();
578 }
579 } finally {
580 if (traceScope != null) traceScope.close();
581 }
582 }
583
584
585
586
587
588 public Stat setAcl(String path, List<ACL> acls, int version)
589 throws KeeperException, InterruptedException {
590 TraceScope traceScope = null;
591 try {
592 traceScope = Trace.startSpan("RecoverableZookeeper.setAcl");
593 RetryCounter retryCounter = retryCounterFactory.create();
594 while (true) {
595 try {
596 return checkZk().setACL(path, acls, version);
597 } catch (KeeperException e) {
598 switch (e.code()) {
599 case CONNECTIONLOSS:
600 case OPERATIONTIMEOUT:
601 retryOrThrow(retryCounter, e, "setAcl");
602 break;
603 case AUTHFAILED:
604 reconnectAfterAuthFailure();
605 retryOrThrow(retryCounter, e, "setAcl");
606 break;
607
608 default:
609 throw e;
610 }
611 }
612 retryCounter.sleepUntilNextRetry();
613 }
614 } finally {
615 if (traceScope != null) traceScope.close();
616 }
617 }
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634 public String create(String path, byte[] data, List<ACL> acl,
635 CreateMode createMode)
636 throws KeeperException, InterruptedException {
637 TraceScope traceScope = null;
638 try {
639 traceScope = Trace.startSpan("RecoverableZookeeper.create");
640 byte[] newData = appendMetaData(data);
641 switch (createMode) {
642 case EPHEMERAL:
643 case PERSISTENT:
644 return createNonSequential(path, newData, acl, createMode);
645
646 case EPHEMERAL_SEQUENTIAL:
647 case PERSISTENT_SEQUENTIAL:
648 return createSequential(path, newData, acl, createMode);
649
650 default:
651 throw new IllegalArgumentException("Unrecognized CreateMode: " +
652 createMode);
653 }
654 } finally {
655 if (traceScope != null) traceScope.close();
656 }
657 }
658
659 private String createNonSequential(String path, byte[] data, List<ACL> acl,
660 CreateMode createMode) throws KeeperException, InterruptedException {
661 RetryCounter retryCounter = retryCounterFactory.create();
662 boolean isRetry = false;
663 while (true) {
664 try {
665 return checkZk().create(path, data, acl, createMode);
666 } catch (KeeperException e) {
667 switch (e.code()) {
668 case NODEEXISTS:
669 if (isRetry) {
670
671
672
673 byte[] currentData = checkZk().getData(path, false, null);
674 if (currentData != null &&
675 Bytes.compareTo(currentData, data) == 0) {
676
677 return path;
678 }
679 LOG.error("Node " + path + " already exists with " +
680 Bytes.toStringBinary(currentData) + ", could not write " +
681 Bytes.toStringBinary(data));
682 throw e;
683 }
684 LOG.debug("Node " + path + " already exists");
685 throw e;
686
687 case CONNECTIONLOSS:
688 case OPERATIONTIMEOUT:
689 retryOrThrow(retryCounter, e, "create");
690 break;
691 case AUTHFAILED:
692 reconnectAfterAuthFailure();
693 retryOrThrow(retryCounter, e, "create");
694 break;
695
696 default:
697 throw e;
698 }
699 }
700 retryCounter.sleepUntilNextRetry();
701 isRetry = true;
702 }
703 }
704
705 private String createSequential(String path, byte[] data,
706 List<ACL> acl, CreateMode createMode)
707 throws KeeperException, InterruptedException {
708 RetryCounter retryCounter = retryCounterFactory.create();
709 boolean first = true;
710 String newPath = path+this.identifier;
711 while (true) {
712 try {
713 if (!first) {
714
715 String previousResult = findPreviousSequentialNode(newPath);
716 if (previousResult != null) {
717 return previousResult;
718 }
719 }
720 first = false;
721 return checkZk().create(newPath, data, acl, createMode);
722 } catch (KeeperException e) {
723 switch (e.code()) {
724 case CONNECTIONLOSS:
725 case OPERATIONTIMEOUT:
726 retryOrThrow(retryCounter, e, "create");
727 break;
728 case AUTHFAILED:
729 reconnectAfterAuthFailure();
730 retryOrThrow(retryCounter, e, "create");
731 break;
732
733 default:
734 throw e;
735 }
736 }
737 retryCounter.sleepUntilNextRetry();
738 }
739 }
740
741
742
743
744 private Iterable<Op> prepareZKMulti(Iterable<Op> ops)
745 throws UnsupportedOperationException {
746 if(ops == null) return null;
747
748 List<Op> preparedOps = new LinkedList<Op>();
749 for (Op op : ops) {
750 if (op.getType() == ZooDefs.OpCode.create) {
751 CreateRequest create = (CreateRequest)op.toRequestRecord();
752 preparedOps.add(Op.create(create.getPath(), appendMetaData(create.getData()),
753 create.getAcl(), create.getFlags()));
754 } else if (op.getType() == ZooDefs.OpCode.delete) {
755
756 preparedOps.add(op);
757 } else if (op.getType() == ZooDefs.OpCode.setData) {
758 SetDataRequest setData = (SetDataRequest)op.toRequestRecord();
759 preparedOps.add(Op.setData(setData.getPath(), appendMetaData(setData.getData()),
760 setData.getVersion()));
761 } else {
762 throw new UnsupportedOperationException("Unexpected ZKOp type: " + op.getClass().getName());
763 }
764 }
765 return preparedOps;
766 }
767
768
769
770
771 public List<OpResult> multi(Iterable<Op> ops)
772 throws KeeperException, InterruptedException {
773 TraceScope traceScope = null;
774 try {
775 traceScope = Trace.startSpan("RecoverableZookeeper.multi");
776 RetryCounter retryCounter = retryCounterFactory.create();
777 Iterable<Op> multiOps = prepareZKMulti(ops);
778 while (true) {
779 try {
780 return checkZk().multi(multiOps);
781 } catch (KeeperException e) {
782 switch (e.code()) {
783 case CONNECTIONLOSS:
784 case OPERATIONTIMEOUT:
785 retryOrThrow(retryCounter, e, "multi");
786 break;
787 case AUTHFAILED:
788 reconnectAfterAuthFailure();
789 retryOrThrow(retryCounter, e, "multi");
790 break;
791
792 default:
793 throw e;
794 }
795 }
796 retryCounter.sleepUntilNextRetry();
797 }
798 } finally {
799 if (traceScope != null) traceScope.close();
800 }
801 }
802
803 private String findPreviousSequentialNode(String path)
804 throws KeeperException, InterruptedException {
805 int lastSlashIdx = path.lastIndexOf('/');
806 assert(lastSlashIdx != -1);
807 String parent = path.substring(0, lastSlashIdx);
808 String nodePrefix = path.substring(lastSlashIdx+1);
809
810 List<String> nodes = checkZk().getChildren(parent, false);
811 List<String> matching = filterByPrefix(nodes, nodePrefix);
812 for (String node : matching) {
813 String nodePath = parent + "/" + node;
814 Stat stat = checkZk().exists(nodePath, false);
815 if (stat != null) {
816 return nodePath;
817 }
818 }
819 return null;
820 }
821
822 public byte[] removeMetaData(byte[] data) {
823 if(data == null || data.length == 0) {
824 return data;
825 }
826
827 byte magic = data[0];
828 if(magic != MAGIC) {
829 return data;
830 }
831
832 int idLength = Bytes.toInt(data, ID_LENGTH_OFFSET);
833 int dataLength = data.length-MAGIC_SIZE-ID_LENGTH_SIZE-idLength;
834 int dataOffset = MAGIC_SIZE+ID_LENGTH_SIZE+idLength;
835
836 byte[] newData = new byte[dataLength];
837 System.arraycopy(data, dataOffset, newData, 0, dataLength);
838 return newData;
839 }
840
841 private byte[] appendMetaData(byte[] data) {
842 if(data == null || data.length == 0){
843 return data;
844 }
845 byte[] salt = Bytes.toBytes(salter.nextLong());
846 int idLength = id.length + salt.length;
847 byte[] newData = new byte[MAGIC_SIZE+ID_LENGTH_SIZE+idLength+data.length];
848 int pos = 0;
849 pos = Bytes.putByte(newData, pos, MAGIC);
850 pos = Bytes.putInt(newData, pos, idLength);
851 pos = Bytes.putBytes(newData, pos, id, 0, id.length);
852 pos = Bytes.putBytes(newData, pos, salt, 0, salt.length);
853 pos = Bytes.putBytes(newData, pos, data, 0, data.length);
854 return newData;
855 }
856
857 public synchronized long getSessionId() {
858 return zk == null ? -1 : zk.getSessionId();
859 }
860
861 public synchronized void close() throws InterruptedException {
862 if (zk != null) zk.close();
863 }
864
865 public synchronized States getState() {
866 return zk == null ? null : zk.getState();
867 }
868
869 public synchronized ZooKeeper getZooKeeper() {
870 return zk;
871 }
872
873 public synchronized byte[] getSessionPasswd() {
874 return zk == null ? null : zk.getSessionPasswd();
875 }
876
877 public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException {
878 checkZk().sync(path, cb, ctx);
879 }
880
881
882
883
884
885
886
887
888
889
890 private static List<String> filterByPrefix(List<String> nodes,
891 String... prefixes) {
892 List<String> lockChildren = new ArrayList<String>();
893 for (String child : nodes){
894 for (String prefix : prefixes){
895 if (child.startsWith(prefix)){
896 lockChildren.add(child);
897 break;
898 }
899 }
900 }
901 return lockChildren;
902 }
903
904 public String getIdentifier() {
905 return identifier;
906 }
907 }