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.replication;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Collections;
25 import java.util.HashMap;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.SortedMap;
29 import java.util.SortedSet;
30 import java.util.TreeMap;
31 import java.util.TreeSet;
32 import java.util.UUID;
33 import java.util.concurrent.atomic.AtomicBoolean;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.hbase.Abortable;
39 import org.apache.hadoop.hbase.HConstants;
40 import org.apache.hadoop.hbase.Server;
41 import org.apache.hadoop.hbase.ServerName;
42 import org.apache.hadoop.hbase.replication.regionserver.Replication;
43 import org.apache.hadoop.hbase.util.Bytes;
44 import org.apache.hadoop.hbase.zookeeper.ClusterId;
45 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
46 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
47 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
48 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
49 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
50 import org.apache.zookeeper.KeeperException;
51 import org.apache.zookeeper.KeeperException.ConnectionLossException;
52 import org.apache.zookeeper.KeeperException.SessionExpiredException;
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83 public class ReplicationZookeeper {
84 private static final Log LOG =
85 LogFactory.getLog(ReplicationZookeeper.class);
86
87 private final static String RS_LOCK_ZNODE = "lock";
88
89
90 public static enum PeerState {
91 ENABLED, DISABLED
92 };
93
94
95 private final ZooKeeperWatcher zookeeper;
96
97 private Map<String, ReplicationPeer> peerClusters;
98
99 private String replicationZNode;
100
101 private String peersZNode;
102
103 private String rsZNode;
104
105 private String rsServerNameZnode;
106
107 private String replicationStateNodeName;
108
109 private String peerStateNodeName;
110 private final Configuration conf;
111
112 private AtomicBoolean replicating;
113
114 private String ourClusterKey;
115
116 private Abortable abortable;
117 private ReplicationStatusTracker statusTracker;
118
119
120
121
122
123
124
125 public ReplicationZookeeper(final Abortable abortable, final Configuration conf,
126 final ZooKeeperWatcher zk)
127 throws KeeperException {
128
129 this.conf = conf;
130 this.zookeeper = zk;
131 this.replicating = new AtomicBoolean();
132 setZNodes(abortable);
133 }
134
135
136
137
138
139
140
141
142
143 public ReplicationZookeeper(final Server server, final AtomicBoolean replicating)
144 throws IOException, KeeperException {
145 this.abortable = server;
146 this.zookeeper = server.getZooKeeper();
147 this.conf = server.getConfiguration();
148 this.replicating = replicating;
149 setZNodes(server);
150
151 this.peerClusters = new HashMap<String, ReplicationPeer>();
152 ZKUtil.createWithParents(this.zookeeper,
153 ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
154 this.rsServerNameZnode = ZKUtil.joinZNode(rsZNode, server.getServerName().toString());
155 ZKUtil.createWithParents(this.zookeeper, this.rsServerNameZnode);
156 connectExistingPeers();
157 }
158
159 private void setZNodes(Abortable abortable) throws KeeperException {
160 String replicationZNodeName =
161 conf.get("zookeeper.znode.replication", "replication");
162 String peersZNodeName =
163 conf.get("zookeeper.znode.replication.peers", "peers");
164 this.peerStateNodeName = conf.get(
165 "zookeeper.znode.replication.peers.state", "peer-state");
166 this.replicationStateNodeName =
167 conf.get("zookeeper.znode.replication.state", "state");
168 String rsZNodeName =
169 conf.get("zookeeper.znode.replication.rs", "rs");
170 this.ourClusterKey = ZKUtil.getZooKeeperClusterKey(this.conf);
171 this.replicationZNode =
172 ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
173 this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
174 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
175 this.rsZNode = ZKUtil.joinZNode(replicationZNode, rsZNodeName);
176 ZKUtil.createWithParents(this.zookeeper, this.rsZNode);
177
178
179 this.statusTracker =
180 new ReplicationStatusTracker(this.zookeeper, abortable);
181 statusTracker.start();
182 readReplicationStateZnode();
183 }
184
185 private void connectExistingPeers() throws IOException, KeeperException {
186 List<String> znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
187 if (znodes != null) {
188 for (String z : znodes) {
189 connectToPeer(z);
190 }
191 }
192 }
193
194
195
196
197
198 public List<String> listPeersIdsAndWatch() {
199 List<String> ids = null;
200 try {
201 ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode);
202 } catch (KeeperException e) {
203 this.abortable.abort("Cannot get the list of peers ", e);
204 }
205 return ids;
206 }
207
208
209
210
211
212 public Map<String,String> listPeers() {
213 Map<String,String> peers = new TreeMap<String,String>();
214 List<String> ids = null;
215 try {
216 ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
217 for (String id : ids) {
218 peers.put(id, Bytes.toString(ZKUtil.getData(this.zookeeper,
219 ZKUtil.joinZNode(this.peersZNode, id))));
220 }
221 } catch (KeeperException e) {
222 this.abortable.abort("Cannot get the list of peers ", e);
223 }
224 return peers;
225 }
226
227
228
229
230
231
232 public List<ServerName> getSlavesAddresses(String peerClusterId) {
233 if (this.peerClusters.size() == 0) {
234 return Collections.emptyList();
235 }
236 ReplicationPeer peer = this.peerClusters.get(peerClusterId);
237 if (peer == null) {
238 return Collections.emptyList();
239 }
240
241 List<ServerName> addresses;
242 try {
243 addresses = fetchSlavesAddresses(peer.getZkw());
244 } catch (KeeperException ke) {
245 reconnectPeer(ke, peer);
246 addresses = Collections.emptyList();
247 }
248 peer.setRegionServers(addresses);
249 return peer.getRegionServers();
250 }
251
252
253
254
255
256
257
258 private List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
259 throws KeeperException {
260 return listChildrenAndGetAsServerNames(zkw, zkw.rsZNode);
261 }
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280 public static List<ServerName> listChildrenAndGetAsServerNames(
281 ZooKeeperWatcher zkw, String znode)
282 throws KeeperException {
283 List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
284 if(children == null) {
285 return Collections.emptyList();
286 }
287 List<ServerName> addresses = new ArrayList<ServerName>(children.size());
288 for (String child : children) {
289 addresses.add(ServerName.parseServerName(child));
290 }
291 return addresses;
292 }
293
294
295
296
297
298
299
300 public boolean connectToPeer(String peerId)
301 throws IOException, KeeperException {
302 if (peerClusters == null) {
303 return false;
304 }
305 if (this.peerClusters.containsKey(peerId)) {
306 return false;
307 }
308 ReplicationPeer peer = getPeer(peerId);
309 if (peer == null) {
310 return false;
311 }
312 this.peerClusters.put(peerId, peer);
313 ZKUtil.createWithParents(this.zookeeper, ZKUtil.joinZNode(
314 this.rsServerNameZnode, peerId));
315 LOG.info("Added new peer cluster " + peer.getClusterKey());
316 return true;
317 }
318
319
320
321
322
323
324
325
326 public ReplicationPeer getPeer(String peerId) throws IOException, KeeperException{
327 String znode = ZKUtil.joinZNode(this.peersZNode, peerId);
328 byte [] data = ZKUtil.getData(this.zookeeper, znode);
329 String otherClusterKey = Bytes.toString(data);
330 if (this.ourClusterKey.equals(otherClusterKey)) {
331 LOG.debug("Not connecting to " + peerId + " because it's us");
332 return null;
333 }
334
335 Configuration otherConf = new Configuration(this.conf);
336 try {
337 ZKUtil.applyClusterKeyToConf(otherConf, otherClusterKey);
338 } catch (IOException e) {
339 LOG.error("Can't get peer because:", e);
340 return null;
341 }
342
343 ReplicationPeer peer = new ReplicationPeer(otherConf, peerId,
344 otherClusterKey);
345 peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId));
346 return peer;
347 }
348
349
350
351
352
353 public void setReplicating(boolean newState) throws KeeperException {
354 ZKUtil.createWithParents(this.zookeeper,
355 ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
356 ZKUtil.setData(this.zookeeper,
357 ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName),
358 Bytes.toBytes(Boolean.toString(newState)));
359 }
360
361
362
363
364
365
366
367 public void removePeer(String id) throws IOException {
368 try {
369 if (!peerExists(id)) {
370 throw new IllegalArgumentException("Cannot remove inexisting peer");
371 }
372 ZKUtil.deleteNodeRecursively(this.zookeeper,
373 ZKUtil.joinZNode(this.peersZNode, id));
374 } catch (KeeperException e) {
375 throw new IOException("Unable to remove a peer", e);
376 }
377 }
378
379
380
381
382
383
384
385
386
387 public void addPeer(String id, String clusterKey) throws IOException {
388 try {
389 if (peerExists(id)) {
390 throw new IllegalArgumentException("Cannot add existing peer");
391 }
392 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
393 ZKUtil.createAndWatch(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id),
394 Bytes.toBytes(clusterKey));
395
396
397
398 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, getPeerStateNode(id),
399 Bytes.toBytes(PeerState.ENABLED.name()));
400 } catch (KeeperException e) {
401 throw new IOException("Unable to add peer", e);
402 }
403 }
404
405 private boolean peerExists(String id) throws KeeperException {
406 return ZKUtil.checkExists(this.zookeeper,
407 ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
408 }
409
410
411
412
413
414
415
416
417 public void enablePeer(String id) throws IOException {
418 changePeerState(id, PeerState.ENABLED);
419 LOG.info("peer " + id + " is enabled");
420 }
421
422
423
424
425
426
427
428
429 public void disablePeer(String id) throws IOException {
430 changePeerState(id, PeerState.DISABLED);
431 LOG.info("peer " + id + " is disabled");
432 }
433
434 private void changePeerState(String id, PeerState state) throws IOException {
435 try {
436 if (!peerExists(id)) {
437 throw new IllegalArgumentException("peer " + id + " is not registered");
438 }
439 String peerStateZNode = getPeerStateNode(id);
440 if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
441 ZKUtil.setData(this.zookeeper, peerStateZNode,
442 Bytes.toBytes(state.name()));
443 } else {
444 ZKUtil.createAndWatch(zookeeper, peerStateZNode,
445 Bytes.toBytes(state.name()));
446 }
447 LOG.info("state of the peer " + id + " changed to " + state.name());
448 } catch (KeeperException e) {
449 throw new IOException("Unable to change state of the peer " + id, e);
450 }
451 }
452
453
454
455
456
457
458
459 public PeerState getPeerState(String id) throws KeeperException {
460 byte[] peerStateBytes = ZKUtil
461 .getData(this.zookeeper, getPeerStateNode(id));
462 return PeerState.valueOf(Bytes.toString(peerStateBytes));
463 }
464
465
466
467
468
469
470
471
472
473
474 public boolean getPeerEnabled(String id) {
475 if (!this.peerClusters.containsKey(id)) {
476 throw new IllegalArgumentException("peer " + id + " is not registered");
477 }
478 return this.peerClusters.get(id).getPeerEnabled().get();
479 }
480
481 private String getPeerStateNode(String id) {
482 return ZKUtil.joinZNode(this.peersZNode,
483 ZKUtil.joinZNode(id, this.peerStateNodeName));
484 }
485
486
487
488
489 private void readReplicationStateZnode() {
490 try {
491 this.replicating.set(getReplication());
492 LOG.info("Replication is now " + (this.replicating.get()?
493 "started" : "stopped"));
494 } catch (KeeperException e) {
495 this.abortable.abort("Failed getting data on from " + getRepStateNode(), e);
496 }
497 }
498
499
500
501
502
503
504
505 public boolean getReplication() throws KeeperException {
506 byte [] data = this.statusTracker.getData(false);
507 if (data == null || data.length == 0) {
508 setReplicating(true);
509 return true;
510 }
511 return Boolean.parseBoolean(Bytes.toString(data));
512 }
513
514 private String getRepStateNode() {
515 return ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName);
516 }
517
518
519
520
521
522
523 public void addLogToList(String filename, String peerId)
524 throws KeeperException {
525 String znode = ZKUtil.joinZNode(this.rsServerNameZnode, peerId);
526 znode = ZKUtil.joinZNode(znode, filename);
527 ZKUtil.createWithParents(this.zookeeper, znode);
528 }
529
530
531
532
533
534
535 public void removeLogFromList(String filename, String clusterId) {
536 try {
537 String znode = ZKUtil.joinZNode(rsServerNameZnode, clusterId);
538 znode = ZKUtil.joinZNode(znode, filename);
539 ZKUtil.deleteNode(this.zookeeper, znode);
540 } catch (KeeperException e) {
541 this.abortable.abort("Failed remove from list", e);
542 }
543 }
544
545
546
547
548
549
550
551
552 public void writeReplicationStatus(String filename, String clusterId,
553 long position) {
554 try {
555 String znode = ZKUtil.joinZNode(this.rsServerNameZnode, clusterId);
556 znode = ZKUtil.joinZNode(znode, filename);
557
558 ZKUtil.setData(this.zookeeper, znode,
559 Bytes.toBytes(Long.toString(position)));
560 } catch (KeeperException e) {
561 this.abortable.abort("Writing replication status", e);
562 }
563 }
564
565
566
567
568
569
570 public List<String> getRegisteredRegionServers() {
571 List<String> result = null;
572 try {
573 result = ZKUtil.listChildrenAndWatchThem(
574 this.zookeeper, this.zookeeper.rsZNode);
575 } catch (KeeperException e) {
576 this.abortable.abort("Get list of registered region servers", e);
577 }
578 return result;
579 }
580
581
582
583
584
585
586 public List<String> getListOfReplicators() {
587 List<String> result = null;
588 try {
589 result = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZNode);
590 } catch (KeeperException e) {
591 this.abortable.abort("Get list of replicators", e);
592 }
593 return result;
594 }
595
596
597
598
599
600
601 public List<String> getListPeersForRS(String rs) {
602 String znode = ZKUtil.joinZNode(rsZNode, rs);
603 List<String> result = null;
604 try {
605 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
606 } catch (KeeperException e) {
607 this.abortable.abort("Get list of peers for rs", e);
608 }
609 return result;
610 }
611
612
613
614
615
616
617
618 public List<String> getListHLogsForPeerForRS(String rs, String id) {
619 String znode = ZKUtil.joinZNode(rsZNode, rs);
620 znode = ZKUtil.joinZNode(znode, id);
621 List<String> result = null;
622 try {
623 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
624 } catch (KeeperException e) {
625 this.abortable.abort("Get list of hlogs for peer", e);
626 }
627 return result;
628 }
629
630
631
632
633
634
635 public boolean lockOtherRS(String znode) {
636 try {
637 String parent = ZKUtil.joinZNode(this.rsZNode, znode);
638 if (parent.equals(rsServerNameZnode)) {
639 LOG.warn("Won't lock because this is us, we're dead!");
640 return false;
641 }
642 String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
643 ZKUtil.createAndWatch(this.zookeeper, p, Bytes.toBytes(rsServerNameZnode));
644 } catch (KeeperException e) {
645
646
647
648
649
650 if (e instanceof KeeperException.NoNodeException ||
651 e instanceof KeeperException.NodeExistsException) {
652 LOG.info("Won't transfer the queue," +
653 " another RS took care of it because of: " + e.getMessage());
654 } else {
655 LOG.info("Failed lock other rs", e);
656 }
657 return false;
658 }
659 return true;
660 }
661
662
663
664
665
666
667
668 public SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
669 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
670 String deadRSZnodePath = ZKUtil.joinZNode(rsZNode, znode);
671 List<String> peerIdsToProcess = null;
672 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
673 try {
674 peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
675 if (peerIdsToProcess == null) return queues;
676 for (String peerId : peerIdsToProcess) {
677 String newPeerId = peerId + "-" + znode;
678 String newPeerZnode = ZKUtil.joinZNode(this.rsServerNameZnode, newPeerId);
679
680 String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
681 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
682 if (hlogs == null || hlogs.size() == 0) {
683 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
684 continue;
685 }
686
687 SortedSet<String> logQueue = new TreeSet<String>();
688 queues.put(newPeerId, logQueue);
689 ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
690 listOfOps.add(op);
691
692 for (String hlog : hlogs) {
693 String oldHlogZnode = ZKUtil.joinZNode(oldClusterZnode, hlog);
694 byte[] logOffset = ZKUtil.getData(this.zookeeper, oldHlogZnode);
695 LOG.debug("Creating " + hlog + " with data " + Bytes.toString(logOffset));
696 String newLogZnode = ZKUtil.joinZNode(newPeerZnode, hlog);
697 listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
698
699 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldHlogZnode));
700 logQueue.add(hlog);
701 }
702
703 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
704 }
705
706 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
707 LOG.debug(" The multi list size is: " + listOfOps.size());
708 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
709 LOG.info("Atomically moved the dead regionserver logs. ");
710 } catch (KeeperException e) {
711
712 LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
713 queues.clear();
714 }
715 return queues;
716 }
717
718
719
720
721
722
723
724
725 public SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
726
727
728 SortedMap<String,SortedSet<String>> queues =
729 new TreeMap<String,SortedSet<String>>();
730 try {
731 String nodePath = ZKUtil.joinZNode(rsZNode, znode);
732 List<String> clusters =
733 ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
734
735 if (clusters == null || clusters.size() <= 1) {
736 return queues;
737 }
738
739 clusters.remove(RS_LOCK_ZNODE);
740 for (String cluster : clusters) {
741
742
743
744 String newCluster = cluster+"-"+znode;
745 String newClusterZnode = ZKUtil.joinZNode(rsServerNameZnode, newCluster);
746 String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
747 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
748
749 if (hlogs == null || hlogs.size() == 0) {
750 continue;
751 }
752 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
753 HConstants.EMPTY_BYTE_ARRAY);
754 SortedSet<String> logQueue = new TreeSet<String>();
755 queues.put(newCluster, logQueue);
756 for (String hlog : hlogs) {
757 String z = ZKUtil.joinZNode(clusterPath, hlog);
758 byte [] position = ZKUtil.getData(this.zookeeper, z);
759 LOG.debug("Creating " + hlog + " with data " + Bytes.toString(position));
760 String child = ZKUtil.joinZNode(newClusterZnode, hlog);
761 ZKUtil.createAndWatch(this.zookeeper, child, position);
762 logQueue.add(hlog);
763 }
764 }
765 } catch (KeeperException e) {
766 this.abortable.abort("Copy queues from rs", e);
767 }
768 return queues;
769 }
770
771
772
773
774
775 public void deleteSource(String peerZnode, boolean closeConnection) {
776 try {
777 ZKUtil.deleteNodeRecursively(this.zookeeper,
778 ZKUtil.joinZNode(rsServerNameZnode, peerZnode));
779 if (closeConnection) {
780 this.peerClusters.get(peerZnode).getZkw().close();
781 this.peerClusters.remove(peerZnode);
782 }
783 } catch (KeeperException e) {
784 this.abortable.abort("Failed delete of " + peerZnode, e);
785 }
786 }
787
788
789
790
791
792 public void deleteRsQueues(String znode) {
793 String fullpath = ZKUtil.joinZNode(rsZNode, znode);
794 try {
795 List<String> clusters =
796 ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
797 for (String cluster : clusters) {
798
799 if (cluster.equals(RS_LOCK_ZNODE)) {
800 continue;
801 }
802 String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
803 ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
804 }
805
806 ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
807 } catch (KeeperException e) {
808 if (e instanceof KeeperException.NoNodeException ||
809 e instanceof KeeperException.NotEmptyException) {
810
811
812
813 if (e.getPath().equals(fullpath)) {
814 return;
815 }
816 }
817 this.abortable.abort("Failed delete of " + znode, e);
818 }
819 }
820
821
822
823
824 public void deleteOwnRSZNode() {
825 try {
826 ZKUtil.deleteNodeRecursively(this.zookeeper,
827 this.rsServerNameZnode);
828 } catch (KeeperException e) {
829
830 if (e instanceof KeeperException.SessionExpiredException) {
831 return;
832 }
833 this.abortable.abort("Failed delete of " + this.rsServerNameZnode, e);
834 }
835 }
836
837
838
839
840
841
842
843
844 public long getHLogRepPosition(String peerId, String hlog)
845 throws KeeperException {
846 String clusterZnode = ZKUtil.joinZNode(rsServerNameZnode, peerId);
847 String znode = ZKUtil.joinZNode(clusterZnode, hlog);
848 String data = Bytes.toString(ZKUtil.getData(this.zookeeper, znode));
849 return data == null || data.length() == 0 ? 0 : Long.parseLong(data);
850 }
851
852
853
854
855
856
857
858
859 public UUID getPeerUUID(String peerId) {
860 ReplicationPeer peer = getPeerClusters().get(peerId);
861 UUID peerUUID = null;
862 try {
863 peerUUID = getUUIDForCluster(peer.getZkw());
864 } catch (KeeperException ke) {
865 reconnectPeer(ke, peer);
866 }
867 return peerUUID;
868 }
869
870
871
872
873
874
875
876 public UUID getUUIDForCluster(ZooKeeperWatcher zkw) throws KeeperException {
877 return UUID.fromString(ClusterId.readClusterIdZNode(zkw));
878 }
879
880 private void reconnectPeer(KeeperException ke, ReplicationPeer peer) {
881 if (ke instanceof ConnectionLossException
882 || ke instanceof SessionExpiredException) {
883 LOG.warn(
884 "Lost the ZooKeeper connection for peer " + peer.getClusterKey(),
885 ke);
886 try {
887 peer.reloadZkWatcher();
888 } catch(IOException io) {
889 LOG.warn(
890 "Creation of ZookeeperWatcher failed for peer "
891 + peer.getClusterKey(), io);
892 }
893 }
894 }
895
896 public void registerRegionServerListener(ZooKeeperListener listener) {
897 this.zookeeper.registerListener(listener);
898 }
899
900
901
902
903
904 public Map<String, ReplicationPeer> getPeerClusters() {
905 return this.peerClusters;
906 }
907
908
909
910
911
912
913 public static String getZNodeName(String fullPath) {
914 String[] parts = fullPath.split("/");
915 return parts.length > 0 ? parts[parts.length-1] : "";
916 }
917
918
919
920
921
922 public ZooKeeperWatcher getZookeeperWatcher() {
923 return this.zookeeper;
924 }
925
926
927
928
929
930
931 public String getPeersZNode() {
932 return peersZNode;
933 }
934
935
936
937
938 public class ReplicationStatusTracker extends ZooKeeperNodeTracker {
939 public ReplicationStatusTracker(ZooKeeperWatcher watcher,
940 Abortable abortable) {
941 super(watcher, getRepStateNode(), abortable);
942 }
943
944 @Override
945 public synchronized void nodeDataChanged(String path) {
946 if (path.equals(node)) {
947 super.nodeDataChanged(path);
948 readReplicationStateZnode();
949 }
950 }
951 }
952 }