1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication;
20
21 import com.google.protobuf.InvalidProtocolBufferException;
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.classification.InterfaceAudience;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.Abortable;
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.Server;
29 import org.apache.hadoop.hbase.ServerName;
30 import org.apache.hadoop.hbase.exceptions.DeserializationException;
31 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
32 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
33 import org.apache.hadoop.hbase.util.Bytes;
34 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
35 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
36 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
37 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
38 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
39 import org.apache.zookeeper.KeeperException;
40 import org.apache.zookeeper.KeeperException.ConnectionLossException;
41 import org.apache.zookeeper.KeeperException.NodeExistsException;
42 import org.apache.zookeeper.KeeperException.SessionExpiredException;
43
44 import java.io.Closeable;
45 import java.io.IOException;
46 import java.util.ArrayList;
47 import java.util.Collections;
48 import java.util.HashMap;
49 import java.util.List;
50 import java.util.Map;
51 import java.util.SortedMap;
52 import java.util.SortedSet;
53 import java.util.TreeMap;
54 import java.util.TreeSet;
55 import java.util.UUID;
56 import java.util.concurrent.atomic.AtomicBoolean;
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
85
86
87 @InterfaceAudience.Private
88 public class ReplicationZookeeper implements Closeable {
89 private static final Log LOG =
90 LogFactory.getLog(ReplicationZookeeper.class);
91
92 private final static String RS_LOCK_ZNODE = "lock";
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
110 private String peerStateNodeName;
111 private final Configuration conf;
112
113 private String ourClusterKey;
114
115 private Abortable abortable;
116 private final ReplicationStateInterface replicationState;
117
118
119
120
121
122 public static final byte[] ENABLED_ZNODE_BYTES =
123 toByteArray(ZooKeeperProtos.ReplicationState.State.ENABLED);
124
125
126
127
128 static final byte[] DISABLED_ZNODE_BYTES =
129 toByteArray(ZooKeeperProtos.ReplicationState.State.DISABLED);
130
131
132
133
134
135
136
137 public ReplicationZookeeper(final Abortable abortable, final Configuration conf,
138 final ZooKeeperWatcher zk) throws KeeperException {
139 this.conf = conf;
140 this.zookeeper = zk;
141 setZNodes(abortable);
142 this.replicationState =
143 new ReplicationStateImpl(this.zookeeper, getRepStateNode(), abortable, new AtomicBoolean());
144 }
145
146
147
148
149
150
151
152
153
154 public ReplicationZookeeper(final Server server, final AtomicBoolean replicating)
155 throws IOException, KeeperException {
156 this.abortable = server;
157 this.zookeeper = server.getZooKeeper();
158 this.conf = server.getConfiguration();
159 setZNodes(server);
160
161 this.replicationState =
162 new ReplicationStateImpl(this.zookeeper, getRepStateNode(), server, replicating);
163 this.peerClusters = new HashMap<String, ReplicationPeer>();
164 ZKUtil.createWithParents(this.zookeeper,
165 ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
166 this.rsServerNameZnode = ZKUtil.joinZNode(rsZNode, server.getServerName().toString());
167 ZKUtil.createWithParents(this.zookeeper, this.rsServerNameZnode);
168 connectExistingPeers();
169 }
170
171 private void setZNodes(Abortable abortable) throws KeeperException {
172 String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
173 String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
174 this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
175 this.replicationStateNodeName = conf.get("zookeeper.znode.replication.state", "state");
176 String rsZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
177 this.ourClusterKey = ZKUtil.getZooKeeperClusterKey(this.conf);
178 this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
179 this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
180 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
181 this.rsZNode = ZKUtil.joinZNode(replicationZNode, rsZNodeName);
182 ZKUtil.createWithParents(this.zookeeper, this.rsZNode);
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 byte[] bytes = ZKUtil.getData(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
219 String clusterKey = null;
220 try {
221 clusterKey = parsePeerFrom(bytes);
222 } catch (DeserializationException de) {
223 LOG.warn("Failed parse of clusterid=" + id + " znode content, continuing.");
224 continue;
225 }
226 peers.put(id, clusterKey);
227 }
228 } catch (KeeperException e) {
229 this.abortable.abort("Cannot get the list of peers ", e);
230 }
231 return peers;
232 }
233
234
235
236
237
238
239
240 public List<ServerName> getSlavesAddresses(String peerClusterId) {
241 if (this.peerClusters.size() == 0) {
242 return Collections.emptyList();
243 }
244 ReplicationPeer peer = this.peerClusters.get(peerClusterId);
245 if (peer == null) {
246 return Collections.emptyList();
247 }
248
249 List<ServerName> addresses;
250 try {
251 addresses = fetchSlavesAddresses(peer.getZkw());
252 } catch (KeeperException ke) {
253 reconnectPeer(ke, peer);
254 addresses = Collections.emptyList();
255 }
256 peer.setRegionServers(addresses);
257 return peer.getRegionServers();
258 }
259
260
261
262
263
264
265
266 private List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
267 throws KeeperException {
268 return listChildrenAndGetAsServerNames(zkw, zkw.rsZNode);
269 }
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288 public static List<ServerName> listChildrenAndGetAsServerNames(
289 ZooKeeperWatcher zkw, String znode)
290 throws KeeperException {
291 List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
292 if(children == null) {
293 return Collections.emptyList();
294 }
295 List<ServerName> addresses = new ArrayList<ServerName>(children.size());
296 for (String child : children) {
297 addresses.add(ServerName.parseServerName(child));
298 }
299 return addresses;
300 }
301
302
303
304
305
306
307
308 public boolean connectToPeer(String peerId)
309 throws IOException, KeeperException {
310 if (peerClusters == null) {
311 return false;
312 }
313 if (this.peerClusters.containsKey(peerId)) {
314 return false;
315 }
316 ReplicationPeer peer = getPeer(peerId);
317 if (peer == null) {
318 return false;
319 }
320 this.peerClusters.put(peerId, peer);
321 ZKUtil.createWithParents(this.zookeeper, ZKUtil.joinZNode(
322 this.rsServerNameZnode, peerId));
323 LOG.info("Added new peer cluster " + peer.getClusterKey());
324 return true;
325 }
326
327
328
329
330
331
332
333
334 public ReplicationPeer getPeer(String peerId) throws IOException, KeeperException{
335 String znode = ZKUtil.joinZNode(this.peersZNode, peerId);
336 byte [] data = ZKUtil.getData(this.zookeeper, znode);
337 String otherClusterKey = "";
338 try {
339 otherClusterKey = parsePeerFrom(data);
340 } catch (DeserializationException e) {
341 LOG.warn("Failed parse of cluster key from peerId=" + peerId
342 + ", specifically the content from the following znode: " + znode);
343 }
344 if (this.ourClusterKey.equals(otherClusterKey)) {
345 LOG.debug("Not connecting to " + peerId + " because it's us");
346 return null;
347 }
348
349 Configuration otherConf = new Configuration(this.conf);
350 try {
351 ZKUtil.applyClusterKeyToConf(otherConf, otherClusterKey);
352 } catch (IOException e) {
353 LOG.error("Can't get peer because:", e);
354 return null;
355 }
356
357 ReplicationPeer peer = new ReplicationPeer(otherConf, peerId,
358 otherClusterKey);
359 peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId));
360 return peer;
361 }
362
363
364
365
366
367
368
369 public void removePeer(String id) throws IOException {
370 try {
371 if (!peerExists(id)) {
372 throw new IllegalArgumentException("Cannot remove inexisting peer");
373 }
374 ZKUtil.deleteNodeRecursively(this.zookeeper,
375 ZKUtil.joinZNode(this.peersZNode, id));
376 } catch (KeeperException e) {
377 throw new IOException("Unable to remove a peer", e);
378 }
379 }
380
381
382
383
384
385
386
387
388
389 public void addPeer(String id, String clusterKey) throws IOException {
390 try {
391 if (peerExists(id)) {
392 throw new IllegalArgumentException("Cannot add existing peer");
393 }
394 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
395 ZKUtil.createAndWatch(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id),
396 toByteArray(clusterKey));
397
398
399
400 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, getPeerStateNode(id),
401 ENABLED_ZNODE_BYTES);
402
403 } catch (KeeperException e) {
404 throw new IOException("Unable to add peer", e);
405 }
406 }
407
408
409
410
411
412
413
414 static byte[] toByteArray(final String clusterKey) {
415 byte[] bytes = ZooKeeperProtos.ReplicationPeer.newBuilder().setClusterkey(clusterKey).build()
416 .toByteArray();
417 return ProtobufUtil.prependPBMagic(bytes);
418 }
419
420
421
422
423
424
425
426
427
428
429 static byte[] toByteArray(final ZooKeeperProtos.ReplicationState.State state) {
430 byte[] bytes = ZooKeeperProtos.ReplicationState.newBuilder().setState(state).build()
431 .toByteArray();
432 return ProtobufUtil.prependPBMagic(bytes);
433 }
434
435
436
437
438
439
440
441 static byte[] toByteArray(
442 final long position) {
443 byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
444 .build().toByteArray();
445 return ProtobufUtil.prependPBMagic(bytes);
446 }
447
448
449
450
451
452
453
454 static byte[] lockToByteArray(
455 final String lockOwner) {
456 byte[] bytes = ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build()
457 .toByteArray();
458 return ProtobufUtil.prependPBMagic(bytes);
459 }
460
461
462
463
464
465
466 static String parsePeerFrom(final byte[] bytes) throws DeserializationException {
467 if (ProtobufUtil.isPBMagicPrefix(bytes)) {
468 int pblen = ProtobufUtil.lengthOfPBMagic();
469 ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer
470 .newBuilder();
471 ZooKeeperProtos.ReplicationPeer peer;
472 try {
473 peer = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
474 } catch (InvalidProtocolBufferException e) {
475 throw new DeserializationException(e);
476 }
477 return peer.getClusterkey();
478 } else {
479 if (bytes.length > 0) {
480 return Bytes.toString(bytes);
481 }
482 return "";
483 }
484 }
485
486
487
488
489
490
491 static ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
492 throws DeserializationException {
493 ProtobufUtil.expectPBMagicPrefix(bytes);
494 int pblen = ProtobufUtil.lengthOfPBMagic();
495 ZooKeeperProtos.ReplicationState.Builder builder = ZooKeeperProtos.ReplicationState
496 .newBuilder();
497 ZooKeeperProtos.ReplicationState state;
498 try {
499 state = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
500 return state.getState();
501 } catch (InvalidProtocolBufferException e) {
502 throw new DeserializationException(e);
503 }
504 }
505
506
507
508
509
510
511 static long parseHLogPositionFrom(
512 final byte[] bytes) throws DeserializationException {
513 if (ProtobufUtil.isPBMagicPrefix(bytes)) {
514 int pblen = ProtobufUtil.lengthOfPBMagic();
515 ZooKeeperProtos.ReplicationHLogPosition.Builder builder = ZooKeeperProtos.ReplicationHLogPosition
516 .newBuilder();
517 ZooKeeperProtos.ReplicationHLogPosition position;
518 try {
519 position = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
520 } catch (InvalidProtocolBufferException e) {
521 throw new DeserializationException(e);
522 }
523 return position.getPosition();
524 } else {
525 if (bytes.length > 0) {
526 return Bytes.toLong(bytes);
527 }
528 return 0;
529 }
530 }
531
532
533
534
535
536
537 static String parseLockOwnerFrom(
538 final byte[] bytes) throws DeserializationException {
539 if (ProtobufUtil.isPBMagicPrefix(bytes)) {
540 int pblen = ProtobufUtil.lengthOfPBMagic();
541 ZooKeeperProtos.ReplicationLock.Builder builder = ZooKeeperProtos.ReplicationLock
542 .newBuilder();
543 ZooKeeperProtos.ReplicationLock lock;
544 try {
545 lock = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
546 } catch (InvalidProtocolBufferException e) {
547 throw new DeserializationException(e);
548 }
549 return lock.getLockOwner();
550 } else {
551 if (bytes.length > 0) {
552 return Bytes.toString(bytes);
553 }
554 return "";
555 }
556 }
557
558 private boolean peerExists(String id) throws KeeperException {
559 return ZKUtil.checkExists(this.zookeeper,
560 ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
561 }
562
563
564
565
566
567
568
569
570 public void enablePeer(String id) throws IOException {
571 changePeerState(id, ZooKeeperProtos.ReplicationState.State.ENABLED);
572 LOG.info("peer " + id + " is enabled");
573 }
574
575
576
577
578
579
580
581
582 public void disablePeer(String id) throws IOException {
583 changePeerState(id, ZooKeeperProtos.ReplicationState.State.DISABLED);
584 LOG.info("peer " + id + " is disabled");
585 }
586
587 private void changePeerState(String id, ZooKeeperProtos.ReplicationState.State state)
588 throws IOException {
589 try {
590 if (!peerExists(id)) {
591 throw new IllegalArgumentException("peer " + id + " is not registered");
592 }
593 String peerStateZNode = getPeerStateNode(id);
594 byte[] stateBytes = (state == ZooKeeperProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
595 : DISABLED_ZNODE_BYTES;
596 if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
597 ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes);
598 } else {
599 ZKUtil.createAndWatch(zookeeper, peerStateZNode, stateBytes);
600 }
601 LOG.info("state of the peer " + id + " changed to " + state.name());
602 } catch (KeeperException e) {
603 throw new IOException("Unable to change state of the peer " + id, e);
604 }
605 }
606
607
608
609
610
611
612
613
614
615
616 public boolean getPeerEnabled(String id) {
617 if (!this.peerClusters.containsKey(id)) {
618 throw new IllegalArgumentException("peer " + id + " is not registered");
619 }
620 return this.peerClusters.get(id).getPeerEnabled().get();
621 }
622
623 private String getPeerStateNode(String id) {
624 return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
625 }
626
627 private String getRepStateNode() {
628 return ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName);
629 }
630
631
632
633
634
635
636
637 public boolean getReplication() throws KeeperException {
638 return this.replicationState.getState();
639 }
640
641
642
643
644
645
646 public void setReplication(boolean newState) throws KeeperException {
647 this.replicationState.setState(newState);
648 }
649
650
651
652
653
654
655 public void addLogToList(String filename, String peerId)
656 throws KeeperException {
657 String znode = ZKUtil.joinZNode(this.rsServerNameZnode, peerId);
658 znode = ZKUtil.joinZNode(znode, filename);
659 ZKUtil.createWithParents(this.zookeeper, znode);
660 }
661
662
663
664
665
666
667 public void removeLogFromList(String filename, String clusterId) {
668 try {
669 String znode = ZKUtil.joinZNode(rsServerNameZnode, clusterId);
670 znode = ZKUtil.joinZNode(znode, filename);
671 ZKUtil.deleteNode(this.zookeeper, znode);
672 } catch (KeeperException e) {
673 this.abortable.abort("Failed remove from list", e);
674 }
675 }
676
677
678
679
680
681
682
683
684 public void writeReplicationStatus(String filename, String clusterId,
685 long position) {
686 try {
687 String znode = ZKUtil.joinZNode(this.rsServerNameZnode, clusterId);
688 znode = ZKUtil.joinZNode(znode, filename);
689
690 ZKUtil.setData(this.zookeeper, znode, toByteArray(position));
691 } catch (KeeperException e) {
692 this.abortable.abort("Writing replication status", e);
693 }
694 }
695
696
697
698
699
700
701 public List<String> getRegisteredRegionServers() {
702 List<String> result = null;
703 try {
704 result = ZKUtil.listChildrenAndWatchThem(
705 this.zookeeper, this.zookeeper.rsZNode);
706 } catch (KeeperException e) {
707 this.abortable.abort("Get list of registered region servers", e);
708 }
709 return result;
710 }
711
712
713
714
715
716
717 public List<String> getListOfReplicators() {
718 List<String> result = null;
719 try {
720 result = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZNode);
721 } catch (KeeperException e) {
722 this.abortable.abort("Get list of replicators", e);
723 }
724 return result;
725 }
726
727
728
729
730
731
732 public List<String> getListPeersForRS(String rs) {
733 String znode = ZKUtil.joinZNode(rsZNode, rs);
734 List<String> result = null;
735 try {
736 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
737 } catch (KeeperException e) {
738 this.abortable.abort("Get list of peers for rs", e);
739 }
740 return result;
741 }
742
743
744
745
746
747
748
749 public List<String> getListHLogsForPeerForRS(String rs, String id) {
750 String znode = ZKUtil.joinZNode(rsZNode, rs);
751 znode = ZKUtil.joinZNode(znode, id);
752 List<String> result = null;
753 try {
754 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
755 } catch (KeeperException e) {
756 this.abortable.abort("Get list of hlogs for peer", e);
757 }
758 return result;
759 }
760
761
762
763
764
765
766 public boolean lockOtherRS(String znode) {
767 try {
768 String parent = ZKUtil.joinZNode(this.rsZNode, znode);
769 if (parent.equals(rsServerNameZnode)) {
770 LOG.warn("Won't lock because this is us, we're dead!");
771 return false;
772 }
773 String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
774 ZKUtil.createAndWatch(this.zookeeper, p, lockToByteArray(rsServerNameZnode));
775 } catch (KeeperException e) {
776
777
778
779
780
781 if (e instanceof KeeperException.NoNodeException ||
782 e instanceof KeeperException.NodeExistsException) {
783 LOG.info("Won't transfer the queue," +
784 " another RS took care of it because of: " + e.getMessage());
785 } else {
786 LOG.info("Failed lock other rs", e);
787 }
788 return false;
789 }
790 return true;
791 }
792
793
794
795
796
797
798
799 public SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
800 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
801 String deadRSZnodePath = ZKUtil.joinZNode(rsZNode, znode);
802 List<String> peerIdsToProcess = null;
803 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
804 try {
805 peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
806 if (peerIdsToProcess == null) return queues;
807 for (String peerId : peerIdsToProcess) {
808 String newPeerId = peerId + "-" + znode;
809 String newPeerZnode = ZKUtil.joinZNode(this.rsServerNameZnode, newPeerId);
810
811 String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
812 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
813 if (hlogs == null || hlogs.size() == 0) {
814 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
815 continue;
816 }
817
818 SortedSet<String> logQueue = new TreeSet<String>();
819 queues.put(newPeerId, logQueue);
820 ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
821 listOfOps.add(op);
822
823 for (String hlog : hlogs) {
824 String oldHlogZnode = ZKUtil.joinZNode(oldClusterZnode, hlog);
825 byte[] logOffset = ZKUtil.getData(this.zookeeper, oldHlogZnode);
826 LOG.debug("Creating " + hlog + " with data " + Bytes.toString(logOffset));
827 String newLogZnode = ZKUtil.joinZNode(newPeerZnode, hlog);
828 listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
829
830 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldHlogZnode));
831 logQueue.add(hlog);
832 }
833
834 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
835 }
836
837 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
838 LOG.debug(" The multi list size is: " + listOfOps.size());
839 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
840 LOG.info("Atomically moved the dead regionserver logs. ");
841 } catch (KeeperException e) {
842
843 LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
844 queues.clear();
845 }
846 return queues;
847 }
848
849
850
851
852
853
854
855
856 public SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
857
858
859 SortedMap<String,SortedSet<String>> queues =
860 new TreeMap<String,SortedSet<String>>();
861 try {
862 String nodePath = ZKUtil.joinZNode(rsZNode, znode);
863 List<String> clusters =
864 ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
865
866 if (clusters == null || clusters.size() <= 1) {
867 return queues;
868 }
869
870 clusters.remove(RS_LOCK_ZNODE);
871 for (String cluster : clusters) {
872
873
874
875 String newCluster = cluster+"-"+znode;
876 String newClusterZnode = ZKUtil.joinZNode(rsServerNameZnode, newCluster);
877 String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
878 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
879
880 if (hlogs == null || hlogs.size() == 0) {
881 continue;
882 }
883 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
884 HConstants.EMPTY_BYTE_ARRAY);
885 SortedSet<String> logQueue = new TreeSet<String>();
886 queues.put(newCluster, logQueue);
887 for (String hlog : hlogs) {
888 String z = ZKUtil.joinZNode(clusterPath, hlog);
889 byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
890 long position = 0;
891 try {
892 position = parseHLogPositionFrom(positionBytes);
893 } catch (DeserializationException e) {
894 LOG.warn("Failed parse of hlog position from the following znode: " + z);
895 }
896 LOG.debug("Creating " + hlog + " with data " + position);
897 String child = ZKUtil.joinZNode(newClusterZnode, hlog);
898
899
900 ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
901 logQueue.add(hlog);
902 }
903 }
904 } catch (KeeperException e) {
905 this.abortable.abort("Copy queues from rs", e);
906 }
907 return queues;
908 }
909
910
911
912
913
914 public void deleteSource(String peerZnode, boolean closeConnection) {
915 try {
916 ZKUtil.deleteNodeRecursively(this.zookeeper,
917 ZKUtil.joinZNode(rsServerNameZnode, peerZnode));
918 if (closeConnection) {
919 this.peerClusters.get(peerZnode).getZkw().close();
920 this.peerClusters.remove(peerZnode);
921 }
922 } catch (KeeperException e) {
923 this.abortable.abort("Failed delete of " + peerZnode, e);
924 }
925 }
926
927
928
929
930
931 public void deleteRsQueues(String znode) {
932 String fullpath = ZKUtil.joinZNode(rsZNode, znode);
933 try {
934 List<String> clusters =
935 ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
936 for (String cluster : clusters) {
937
938 if (cluster.equals(RS_LOCK_ZNODE)) {
939 continue;
940 }
941 String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
942 ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
943 }
944
945 ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
946 } catch (KeeperException e) {
947 if (e instanceof KeeperException.NoNodeException ||
948 e instanceof KeeperException.NotEmptyException) {
949
950
951
952 if (e.getPath().equals(fullpath)) {
953 return;
954 }
955 }
956 this.abortable.abort("Failed delete of " + znode, e);
957 }
958 }
959
960
961
962
963 public void deleteOwnRSZNode() {
964 try {
965 ZKUtil.deleteNodeRecursively(this.zookeeper,
966 this.rsServerNameZnode);
967 } catch (KeeperException e) {
968
969 if (e instanceof KeeperException.SessionExpiredException) {
970 return;
971 }
972 this.abortable.abort("Failed delete of " + this.rsServerNameZnode, e);
973 }
974 }
975
976
977
978
979
980
981
982
983 public long getHLogRepPosition(String peerId, String hlog)
984 throws KeeperException {
985 String clusterZnode = ZKUtil.joinZNode(rsServerNameZnode, peerId);
986 String znode = ZKUtil.joinZNode(clusterZnode, hlog);
987 byte[] bytes = ZKUtil.getData(this.zookeeper, znode);
988 try {
989 return parseHLogPositionFrom(bytes);
990 } catch (DeserializationException de) {
991 LOG.warn("Failed parse of HLogPosition for peerId=" + peerId + " and hlog=" + hlog
992 + "znode content, continuing.");
993 }
994
995
996 return 0;
997 }
998
999
1000
1001
1002
1003
1004
1005
1006 public UUID getPeerUUID(String peerId) {
1007 ReplicationPeer peer = getPeerClusters().get(peerId);
1008 UUID peerUUID = null;
1009 try {
1010 peerUUID = getUUIDForCluster(peer.getZkw());
1011 } catch (KeeperException ke) {
1012 reconnectPeer(ke, peer);
1013 }
1014 return peerUUID;
1015 }
1016
1017
1018
1019
1020
1021
1022
1023 public UUID getUUIDForCluster(ZooKeeperWatcher zkw) throws KeeperException {
1024 return UUID.fromString(ZKClusterId.readClusterIdZNode(zkw));
1025 }
1026
1027 private void reconnectPeer(KeeperException ke, ReplicationPeer peer) {
1028 if (ke instanceof ConnectionLossException
1029 || ke instanceof SessionExpiredException) {
1030 LOG.warn(
1031 "Lost the ZooKeeper connection for peer " + peer.getClusterKey(),
1032 ke);
1033 try {
1034 peer.reloadZkWatcher();
1035 } catch(IOException io) {
1036 LOG.warn(
1037 "Creation of ZookeeperWatcher failed for peer "
1038 + peer.getClusterKey(), io);
1039 }
1040 }
1041 }
1042
1043 public void registerRegionServerListener(ZooKeeperListener listener) {
1044 this.zookeeper.registerListener(listener);
1045 }
1046
1047
1048
1049
1050
1051 public Map<String, ReplicationPeer> getPeerClusters() {
1052 return this.peerClusters;
1053 }
1054
1055
1056
1057
1058
1059
1060 public boolean isPeerPath(String path) {
1061 return path.split("/").length == peersZNode.split("/").length + 1;
1062 }
1063
1064
1065
1066
1067
1068
1069 public static String getZNodeName(String fullPath) {
1070 String[] parts = fullPath.split("/");
1071 return parts.length > 0 ? parts[parts.length-1] : "";
1072 }
1073
1074
1075
1076
1077
1078 public ZooKeeperWatcher getZookeeperWatcher() {
1079 return this.zookeeper;
1080 }
1081
1082
1083
1084
1085
1086
1087 public String getPeersZNode() {
1088 return peersZNode;
1089 }
1090
1091 @Override
1092 public void close() throws IOException {
1093 if (replicationState != null) replicationState.close();
1094 }
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105 static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path)
1106 throws NodeExistsException, KeeperException {
1107 if (ZKUtil.checkExists(zookeeper, path) == -1) {
1108
1109
1110
1111 ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path, ENABLED_ZNODE_BYTES);
1112 return true;
1113 }
1114 return false;
1115 }
1116
1117
1118
1119
1120
1121
1122
1123 static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
1124 ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
1125 return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
1126 }
1127 }