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