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