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.concurrent.atomic.AtomicBoolean;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.conf.Configuration;
37 import org.apache.hadoop.hbase.Abortable;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.Server;
40 import org.apache.hadoop.hbase.ServerName;
41 import org.apache.hadoop.hbase.util.Bytes;
42 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
43 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
44 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
45 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
46 import org.apache.zookeeper.KeeperException;
47 import org.apache.zookeeper.KeeperException.ConnectionLossException;
48 import org.apache.zookeeper.KeeperException.SessionExpiredException;
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
74
75
76
77 public class ReplicationZookeeper {
78 private static final Log LOG =
79 LogFactory.getLog(ReplicationZookeeper.class);
80
81 private final static String RS_LOCK_ZNODE = "lock";
82
83 private final ZooKeeperWatcher zookeeper;
84
85 private Map<String, ReplicationPeer> peerClusters;
86
87 private String replicationZNode;
88
89 private String peersZNode;
90
91 private String rsZNode;
92
93 private String rsServerNameZnode;
94
95 private String replicationStateNodeName;
96 private final Configuration conf;
97
98 private AtomicBoolean replicating;
99
100 private String ourClusterKey;
101
102 private Abortable abortable;
103 private ReplicationStatusTracker statusTracker;
104
105
106
107
108
109
110
111 public ReplicationZookeeper(final Abortable abortable, final Configuration conf,
112 final ZooKeeperWatcher zk)
113 throws KeeperException {
114
115 this.conf = conf;
116 this.zookeeper = zk;
117 this.replicating = new AtomicBoolean();
118 setZNodes(abortable);
119 }
120
121
122
123
124
125
126
127
128
129 public ReplicationZookeeper(final Server server, final AtomicBoolean replicating)
130 throws IOException, KeeperException {
131 this.abortable = server;
132 this.zookeeper = server.getZooKeeper();
133 this.conf = server.getConfiguration();
134 this.replicating = replicating;
135 setZNodes(server);
136
137 this.peerClusters = new HashMap<String, ReplicationPeer>();
138 ZKUtil.createWithParents(this.zookeeper,
139 ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
140 this.rsServerNameZnode = ZKUtil.joinZNode(rsZNode, server.getServerName().toString());
141 ZKUtil.createWithParents(this.zookeeper, this.rsServerNameZnode);
142 connectExistingPeers();
143 }
144
145 private void setZNodes(Abortable abortable) throws KeeperException {
146 String replicationZNodeName =
147 conf.get("zookeeper.znode.replication", "replication");
148 String peersZNodeName =
149 conf.get("zookeeper.znode.replication.peers", "peers");
150 this.replicationStateNodeName =
151 conf.get("zookeeper.znode.replication.state", "state");
152 String rsZNodeName =
153 conf.get("zookeeper.znode.replication.rs", "rs");
154 this.ourClusterKey = ZKUtil.getZooKeeperClusterKey(this.conf);
155 this.replicationZNode =
156 ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
157 this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
158 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
159 this.rsZNode = ZKUtil.joinZNode(replicationZNode, rsZNodeName);
160 ZKUtil.createWithParents(this.zookeeper, this.rsZNode);
161
162
163 this.statusTracker =
164 new ReplicationStatusTracker(this.zookeeper, abortable);
165 statusTracker.start();
166 readReplicationStateZnode();
167 }
168
169 private void connectExistingPeers() throws IOException, KeeperException {
170 List<String> znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
171 if (znodes != null) {
172 for (String z : znodes) {
173 connectToPeer(z);
174 }
175 }
176 }
177
178
179
180
181
182 public List<String> listPeersIdsAndWatch() {
183 List<String> ids = null;
184 try {
185 ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode);
186 } catch (KeeperException e) {
187 this.abortable.abort("Cannot get the list of peers ", e);
188 }
189 return ids;
190 }
191
192
193
194
195
196 public Map<String,String> listPeers() {
197 Map<String,String> peers = new TreeMap<String,String>();
198 List<String> ids = null;
199 try {
200 ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
201 for (String id : ids) {
202 peers.put(id, Bytes.toString(ZKUtil.getData(this.zookeeper,
203 ZKUtil.joinZNode(this.peersZNode, id))));
204 }
205 } catch (KeeperException e) {
206 this.abortable.abort("Cannot get the list of peers ", e);
207 }
208 return peers;
209 }
210
211
212
213
214
215
216 public List<ServerName> getSlavesAddresses(String peerClusterId) {
217 if (this.peerClusters.size() == 0) {
218 return new ArrayList<ServerName>(0);
219 }
220 ReplicationPeer peer = this.peerClusters.get(peerClusterId);
221 if (peer == null) {
222 return new ArrayList<ServerName>(0);
223 }
224
225 List<ServerName> addresses;
226 try {
227 addresses = fetchSlavesAddresses(peer.getZkw());
228 } catch (KeeperException ke) {
229 if (ke instanceof ConnectionLossException
230 || ke instanceof SessionExpiredException) {
231 LOG.warn(
232 "Lost the ZooKeeper connection for peer " + peer.getClusterKey(),
233 ke);
234 try {
235 peer.reloadZkWatcher();
236 } catch(IOException io) {
237 LOG.warn(
238 "Creation of ZookeeperWatcher failed for peer "
239 + peer.getClusterKey(), io);
240 }
241 }
242 addresses = Collections.emptyList();
243 }
244 peer.setRegionServers(addresses);
245 return peer.getRegionServers();
246 }
247
248
249
250
251
252
253
254 private List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
255 throws KeeperException {
256 return listChildrenAndGetAsServerNames(zkw, zkw.rsZNode);
257 }
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276 public static List<ServerName> listChildrenAndGetAsServerNames(
277 ZooKeeperWatcher zkw, String znode)
278 throws KeeperException {
279 List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
280 if(children == null) {
281 return null;
282 }
283 List<ServerName> addresses = new ArrayList<ServerName>(children.size());
284 for (String child : children) {
285 addresses.add(ServerName.parseServerName(child));
286 }
287 return addresses;
288 }
289
290
291
292
293
294
295
296 public boolean connectToPeer(String peerId)
297 throws IOException, KeeperException {
298 if (peerClusters == null) {
299 return false;
300 }
301 if (this.peerClusters.containsKey(peerId)) {
302 return false;
303 }
304 ReplicationPeer peer = getPeer(peerId);
305 if (peer == null) {
306 return false;
307 }
308 this.peerClusters.put(peerId, peer);
309 ZKUtil.createWithParents(this.zookeeper, ZKUtil.joinZNode(
310 this.rsServerNameZnode, peerId));
311 LOG.info("Added new peer cluster " + peer.getClusterKey());
312 return true;
313 }
314
315
316
317
318
319
320
321
322 public ReplicationPeer getPeer(String peerId) throws IOException, KeeperException{
323 String znode = ZKUtil.joinZNode(this.peersZNode, peerId);
324 byte [] data = ZKUtil.getData(this.zookeeper, znode);
325 String otherClusterKey = Bytes.toString(data);
326 if (this.ourClusterKey.equals(otherClusterKey)) {
327 LOG.debug("Not connecting to " + peerId + " because it's us");
328 return null;
329 }
330
331 Configuration otherConf = new Configuration(this.conf);
332 try {
333 ZKUtil.applyClusterKeyToConf(otherConf, otherClusterKey);
334 } catch (IOException e) {
335 LOG.error("Can't get peer because:", e);
336 return null;
337 }
338
339 return new ReplicationPeer(otherConf, peerId,
340 otherClusterKey);
341 }
342
343
344
345
346
347 public void setReplicating(boolean newState) throws KeeperException {
348 ZKUtil.createWithParents(this.zookeeper,
349 ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
350 ZKUtil.setData(this.zookeeper,
351 ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName),
352 Bytes.toBytes(Boolean.toString(newState)));
353 }
354
355
356
357
358
359
360
361 public void removePeer(String id) throws IOException {
362 try {
363 if (!peerExists(id)) {
364 throw new IllegalArgumentException("Cannot remove inexisting peer");
365 }
366 ZKUtil.deleteNode(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
367 } catch (KeeperException e) {
368 throw new IOException("Unable to remove a peer", e);
369 }
370 }
371
372
373
374
375
376
377
378
379
380 public void addPeer(String id, String clusterKey) throws IOException {
381 try {
382 if (peerExists(id)) {
383 throw new IllegalArgumentException("Cannot add existing peer");
384 }
385 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
386 ZKUtil.createAndWatch(this.zookeeper,
387 ZKUtil.joinZNode(this.peersZNode, id), Bytes.toBytes(clusterKey));
388 } catch (KeeperException e) {
389 throw new IOException("Unable to add peer", e);
390 }
391 }
392
393 private boolean peerExists(String id) throws KeeperException {
394 return ZKUtil.checkExists(this.zookeeper,
395 ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
396 }
397
398
399
400
401 private void readReplicationStateZnode() {
402 try {
403 this.replicating.set(getReplication());
404 LOG.info("Replication is now " + (this.replicating.get()?
405 "started" : "stopped"));
406 } catch (KeeperException e) {
407 this.abortable.abort("Failed getting data on from " + getRepStateNode(), e);
408 }
409 }
410
411
412
413
414
415
416
417 public boolean getReplication() throws KeeperException {
418 byte [] data = this.statusTracker.getData(false);
419 if (data == null || data.length == 0) {
420 setReplicating(true);
421 return true;
422 }
423 return Boolean.parseBoolean(Bytes.toString(data));
424 }
425
426 private String getRepStateNode() {
427 return ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName);
428 }
429
430
431
432
433
434
435 public void addLogToList(String filename, String peerId)
436 throws KeeperException{
437 String znode = ZKUtil.joinZNode(this.rsServerNameZnode, peerId);
438 znode = ZKUtil.joinZNode(znode, filename);
439 ZKUtil.createWithParents(this.zookeeper, znode);
440 }
441
442
443
444
445
446
447 public void removeLogFromList(String filename, String clusterId) {
448 try {
449 String znode = ZKUtil.joinZNode(rsServerNameZnode, clusterId);
450 znode = ZKUtil.joinZNode(znode, filename);
451 ZKUtil.deleteNode(this.zookeeper, znode);
452 } catch (KeeperException e) {
453 this.abortable.abort("Failed remove from list", e);
454 }
455 }
456
457
458
459
460
461
462
463
464 public void writeReplicationStatus(String filename, String clusterId,
465 long position) {
466 try {
467 String znode = ZKUtil.joinZNode(this.rsServerNameZnode, clusterId);
468 znode = ZKUtil.joinZNode(znode, filename);
469
470 ZKUtil.setData(this.zookeeper, znode,
471 Bytes.toBytes(Long.toString(position)));
472 } catch (KeeperException e) {
473 this.abortable.abort("Writing replication status", e);
474 }
475 }
476
477
478
479
480
481
482 public List<String> getRegisteredRegionServers() {
483 List<String> result = null;
484 try {
485 result = ZKUtil.listChildrenAndWatchThem(
486 this.zookeeper, this.zookeeper.rsZNode);
487 } catch (KeeperException e) {
488 this.abortable.abort("Get list of registered region servers", e);
489 }
490 return result;
491 }
492
493
494
495
496
497
498 public List<String> getListOfReplicators() {
499 List<String> result = null;
500 try {
501 result = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZNode);
502 } catch (KeeperException e) {
503 this.abortable.abort("Get list of replicators", e);
504 }
505 return result;
506 }
507
508
509
510
511
512
513 public List<String> getListPeersForRS(String rs) {
514 String znode = ZKUtil.joinZNode(rsZNode, rs);
515 List<String> result = null;
516 try {
517 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
518 } catch (KeeperException e) {
519 this.abortable.abort("Get list of peers for rs", e);
520 }
521 return result;
522 }
523
524
525
526
527
528
529
530 public List<String> getListHLogsForPeerForRS(String rs, String id) {
531 String znode = ZKUtil.joinZNode(rsZNode, rs);
532 znode = ZKUtil.joinZNode(znode, id);
533 List<String> result = null;
534 try {
535 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
536 } catch (KeeperException e) {
537 this.abortable.abort("Get list of hlogs for peer", e);
538 }
539 return result;
540 }
541
542
543
544
545
546
547 public boolean lockOtherRS(String znode) {
548 try {
549 String parent = ZKUtil.joinZNode(this.rsZNode, znode);
550 if (parent.equals(rsServerNameZnode)) {
551 LOG.warn("Won't lock because this is us, we're dead!");
552 return false;
553 }
554 String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
555 ZKUtil.createAndWatch(this.zookeeper, p, Bytes.toBytes(rsServerNameZnode));
556 } catch (KeeperException e) {
557
558
559
560
561
562 if (e instanceof KeeperException.NoNodeException ||
563 e instanceof KeeperException.NodeExistsException) {
564 LOG.info("Won't transfer the queue," +
565 " another RS took care of it because of: " + e.getMessage());
566 } else {
567 LOG.info("Failed lock other rs", e);
568 }
569 return false;
570 }
571 return true;
572 }
573
574
575
576
577
578
579
580
581 public SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
582
583
584 SortedMap<String,SortedSet<String>> queues =
585 new TreeMap<String,SortedSet<String>>();
586 try {
587 String nodePath = ZKUtil.joinZNode(rsZNode, znode);
588 List<String> clusters =
589 ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
590
591 if (clusters == null || clusters.size() <= 1) {
592 return queues;
593 }
594
595 clusters.remove(RS_LOCK_ZNODE);
596 for (String cluster : clusters) {
597
598
599
600 String newCluster = cluster+"-"+znode;
601 String newClusterZnode = ZKUtil.joinZNode(rsServerNameZnode, newCluster);
602 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
603 HConstants.EMPTY_BYTE_ARRAY);
604 String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
605 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
606
607 if (hlogs == null || hlogs.size() == 0) {
608 continue;
609 }
610 SortedSet<String> logQueue = new TreeSet<String>();
611 queues.put(newCluster, logQueue);
612 for (String hlog : hlogs) {
613 String z = ZKUtil.joinZNode(clusterPath, hlog);
614 byte [] position = ZKUtil.getData(this.zookeeper, z);
615 LOG.debug("Creating " + hlog + " with data " + Bytes.toString(position));
616 String child = ZKUtil.joinZNode(newClusterZnode, hlog);
617 ZKUtil.createAndWatch(this.zookeeper, child, position);
618 logQueue.add(hlog);
619 }
620 }
621 } catch (KeeperException e) {
622 this.abortable.abort("Copy queues from rs", e);
623 }
624 return queues;
625 }
626
627
628
629
630
631 public void deleteSource(String peerZnode, boolean closeConnection) {
632 try {
633 ZKUtil.deleteNodeRecursively(this.zookeeper,
634 ZKUtil.joinZNode(rsServerNameZnode, peerZnode));
635 if (closeConnection) {
636 this.peerClusters.get(peerZnode).getZkw().close();
637 this.peerClusters.remove(peerZnode);
638 }
639 } catch (KeeperException e) {
640 this.abortable.abort("Failed delete of " + peerZnode, e);
641 }
642 }
643
644
645
646
647
648 public void deleteRsQueues(String znode) {
649 String fullpath = ZKUtil.joinZNode(rsZNode, znode);
650 try {
651 List<String> clusters =
652 ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
653 for (String cluster : clusters) {
654
655 if (cluster.equals(RS_LOCK_ZNODE)) {
656 continue;
657 }
658 String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
659 ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
660 }
661
662 ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
663 } catch (KeeperException e) {
664 if (e instanceof KeeperException.NoNodeException ||
665 e instanceof KeeperException.NotEmptyException) {
666
667
668
669 if (e.getPath().equals(fullpath)) {
670 return;
671 }
672 }
673 this.abortable.abort("Failed delete of " + znode, e);
674 }
675 }
676
677
678
679
680 public void deleteOwnRSZNode() {
681 try {
682 ZKUtil.deleteNodeRecursively(this.zookeeper,
683 this.rsServerNameZnode);
684 } catch (KeeperException e) {
685
686 if (e instanceof KeeperException.SessionExpiredException) {
687 return;
688 }
689 this.abortable.abort("Failed delete of " + this.rsServerNameZnode, e);
690 }
691 }
692
693
694
695
696
697
698
699
700 public long getHLogRepPosition(String peerId, String hlog)
701 throws KeeperException {
702 String clusterZnode = ZKUtil.joinZNode(rsServerNameZnode, peerId);
703 String znode = ZKUtil.joinZNode(clusterZnode, hlog);
704 String data = Bytes.toString(ZKUtil.getData(this.zookeeper, znode));
705 return data == null || data.length() == 0 ? 0 : Long.parseLong(data);
706 }
707
708 public void registerRegionServerListener(ZooKeeperListener listener) {
709 this.zookeeper.registerListener(listener);
710 }
711
712
713
714
715
716 public Map<String, ReplicationPeer> getPeerClusters() {
717 return this.peerClusters;
718 }
719
720
721
722
723
724
725 public static String getZNodeName(String fullPath) {
726 String[] parts = fullPath.split("/");
727 return parts.length > 0 ? parts[parts.length-1] : "";
728 }
729
730
731
732
733
734 public ZooKeeperWatcher getZookeeperWatcher() {
735 return this.zookeeper;
736 }
737
738
739
740
741
742
743 public String getPeersZNode() {
744 return peersZNode;
745 }
746
747
748
749
750 public class ReplicationStatusTracker extends ZooKeeperNodeTracker {
751 public ReplicationStatusTracker(ZooKeeperWatcher watcher,
752 Abortable abortable) {
753 super(watcher, getRepStateNode(), abortable);
754 }
755
756 @Override
757 public synchronized void nodeDataChanged(String path) {
758 if (path.equals(node)) {
759 super.nodeDataChanged(path);
760 readReplicationStateZnode();
761 }
762 }
763 }
764 }