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 java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Collections;
24 import java.util.HashMap;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.Set;
28 import java.util.TreeMap;
29 import java.util.UUID;
30 import java.util.concurrent.ConcurrentHashMap;
31 import java.util.concurrent.ConcurrentMap;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.hbase.Abortable;
37 import org.apache.hadoop.hbase.ServerName;
38 import org.apache.hadoop.hbase.exceptions.DeserializationException;
39 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
40 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
41 import org.apache.hadoop.hbase.util.Bytes;
42 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
43 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
44 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
45 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
46 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
47 import org.apache.zookeeper.KeeperException;
48 import org.apache.zookeeper.KeeperException.AuthFailedException;
49 import org.apache.zookeeper.KeeperException.ConnectionLossException;
50 import org.apache.zookeeper.KeeperException.SessionExpiredException;
51
52 import com.google.protobuf.InvalidProtocolBufferException;
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 public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements ReplicationPeers {
81
82
83 private Map<String, ReplicationPeer> peerClusters;
84 private final String tableCFsNodeName;
85
86 private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class);
87
88 public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf,
89 Abortable abortable) {
90 super(zk, conf, abortable);
91 this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
92 this.peerClusters = new ConcurrentHashMap<String, ReplicationPeer>();
93 }
94
95 @Override
96 public void init() throws ReplicationException {
97 try {
98 if (ZKUtil.checkExists(this.zookeeper, this.peersZNode) < 0) {
99 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
100 }
101 } catch (KeeperException e) {
102 throw new ReplicationException("Could not initialize replication peers", e);
103 }
104 connectExistingPeers();
105 }
106
107 @Override
108 public void addPeer(String id, String clusterKey) throws ReplicationException {
109 addPeer(id, clusterKey, null);
110 }
111
112 @Override
113 public void addPeer(String id, String clusterKey, String tableCFs) throws ReplicationException {
114 try {
115 if (peerExists(id)) {
116 throw new IllegalArgumentException("Cannot add a peer with id=" + id
117 + " because that id already exists.");
118 }
119
120 if(id.contains("-")){
121 throw new IllegalArgumentException("Found invalid peer name:" + id);
122 }
123
124 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
125 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
126 ZKUtilOp op1 =
127 ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
128 toByteArray(clusterKey));
129
130
131
132
133 ZKUtilOp op2 = ZKUtilOp.createAndFailSilent(getPeerStateNode(id), ENABLED_ZNODE_BYTES);
134 String tableCFsStr = (tableCFs == null) ? "" : tableCFs;
135 ZKUtilOp op3 = ZKUtilOp.createAndFailSilent(getTableCFsNode(id), Bytes.toBytes(tableCFsStr));
136 listOfOps.add(op1);
137 listOfOps.add(op2);
138 listOfOps.add(op3);
139 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
140 } catch (KeeperException e) {
141 throw new ReplicationException("Could not add peer with id=" + id + ", clusterKey="
142 + clusterKey, e);
143 }
144 }
145
146 @Override
147 public void removePeer(String id) throws ReplicationException {
148 try {
149 if (!peerExists(id)) {
150 throw new IllegalArgumentException("Cannot remove peer with id=" + id
151 + " because that id does not exist.");
152 }
153 ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
154 } catch (KeeperException e) {
155 throw new ReplicationException("Could not remove peer with id=" + id, e);
156 }
157 }
158
159 @Override
160 public void enablePeer(String id) throws ReplicationException {
161 changePeerState(id, ZooKeeperProtos.ReplicationState.State.ENABLED);
162 LOG.info("peer " + id + " is enabled");
163 }
164
165 @Override
166 public void disablePeer(String id) throws ReplicationException {
167 changePeerState(id, ZooKeeperProtos.ReplicationState.State.DISABLED);
168 LOG.info("peer " + id + " is disabled");
169 }
170
171 @Override
172 public String getPeerTableCFsConfig(String id) throws ReplicationException {
173 try {
174 if (!peerExists(id)) {
175 throw new IllegalArgumentException("peer " + id + " doesn't exist");
176 }
177 try {
178 return Bytes.toString(ZKUtil.getData(this.zookeeper, getTableCFsNode(id)));
179 } catch (Exception e) {
180 throw new ReplicationException(e);
181 }
182 } catch (KeeperException e) {
183 throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id, e);
184 }
185 }
186
187 @Override
188 public void setPeerTableCFsConfig(String id, String tableCFsStr) throws ReplicationException {
189 try {
190 if (!peerExists(id)) {
191 throw new IllegalArgumentException("Cannot set peer tableCFs because id=" + id
192 + " does not exist.");
193 }
194 String tableCFsZKNode = getTableCFsNode(id);
195 byte[] tableCFs = Bytes.toBytes(tableCFsStr);
196 if (ZKUtil.checkExists(this.zookeeper, tableCFsZKNode) != -1) {
197 ZKUtil.setData(this.zookeeper, tableCFsZKNode, tableCFs);
198 } else {
199 ZKUtil.createAndWatch(this.zookeeper, tableCFsZKNode, tableCFs);
200 }
201 LOG.info("Peer tableCFs with id= " + id + " is now " + tableCFsStr);
202 } catch (KeeperException e) {
203 throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e);
204 }
205 }
206
207 @Override
208 public Map<String, List<String>> getTableCFs(String id) throws IllegalArgumentException {
209 ReplicationPeer replicationPeer = this.peerClusters.get(id);
210 if (replicationPeer == null) {
211 throw new IllegalArgumentException("Peer with id= " + id + " is not connected");
212 }
213 return replicationPeer.getTableCFs();
214 }
215
216 @Override
217 public boolean getStatusOfConnectedPeer(String id) {
218 ReplicationPeer replicationPeer = this.peerClusters.get(id);
219 if (replicationPeer == null) {
220 throw new IllegalArgumentException("Peer with id= " + id + " is not connected");
221 }
222 return replicationPeer.getPeerEnabled().get();
223 }
224
225 @Override
226 public boolean getStatusOfPeerFromBackingStore(String id) throws ReplicationException {
227 try {
228 if (!peerExists(id)) {
229 throw new IllegalArgumentException("peer " + id + " doesn't exist");
230 }
231 String peerStateZNode = getPeerStateNode(id);
232 try {
233 return ReplicationPeer.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode));
234 } catch (KeeperException e) {
235 throw new ReplicationException(e);
236 } catch (DeserializationException e) {
237 throw new ReplicationException(e);
238 }
239 } catch (KeeperException e) {
240 throw new ReplicationException("Unable to get status of the peer with id=" + id +
241 " from backing store", e);
242 }
243 }
244
245 @Override
246 public boolean connectToPeer(String peerId) throws ReplicationException {
247 if (peerClusters == null) {
248 return false;
249 }
250 if (this.peerClusters.containsKey(peerId)) {
251 return false;
252 }
253 ReplicationPeer peer = null;
254 try {
255 peer = getPeer(peerId);
256 } catch (Exception e) {
257 throw new ReplicationException("Error connecting to peer with id=" + peerId, e);
258 }
259 if (peer == null) {
260 return false;
261 }
262 ReplicationPeer previous =
263 ((ConcurrentMap<String, ReplicationPeer>) peerClusters).putIfAbsent(peerId, peer);
264 if (previous == null) {
265 LOG.info("Added new peer cluster=" + peer.getClusterKey());
266 } else {
267 LOG.info("Peer already present, " + previous.getClusterKey() + ", new cluster=" +
268 peer.getClusterKey());
269 }
270 return true;
271 }
272
273 @Override
274 public void disconnectFromPeer(String peerId) {
275 ReplicationPeer rp = this.peerClusters.get(peerId);
276 if (rp != null) {
277 rp.getZkw().close();
278 ((ConcurrentMap<String, ReplicationPeer>) peerClusters).remove(peerId, rp);
279 }
280 }
281
282 @Override
283 public Map<String, String> getAllPeerClusterKeys() {
284 Map<String, String> peers = new TreeMap<String, String>();
285 List<String> ids = null;
286 try {
287 ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
288 for (String id : ids) {
289 byte[] bytes = ZKUtil.getData(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
290 String clusterKey = null;
291 try {
292 clusterKey = parsePeerFrom(bytes);
293 } catch (DeserializationException de) {
294 LOG.warn("Failed parse of clusterid=" + id + " znode content, continuing.");
295 continue;
296 }
297 peers.put(id, clusterKey);
298 }
299 } catch (KeeperException e) {
300 this.abortable.abort("Cannot get the list of peers ", e);
301 }
302 return peers;
303 }
304
305 @Override
306 public List<ServerName> getRegionServersOfConnectedPeer(String peerId) {
307 if (this.peerClusters.size() == 0) {
308 return Collections.emptyList();
309 }
310 ReplicationPeer peer = this.peerClusters.get(peerId);
311 if (peer == null) {
312 return Collections.emptyList();
313 }
314
315
316
317
318 synchronized (peer) {
319 List<ServerName> addresses;
320 try {
321 addresses = fetchSlavesAddresses(peer.getZkw());
322 }
323 catch (KeeperException ke) {
324 if (LOG.isDebugEnabled()) {
325 LOG.debug("Fetch salves addresses failed.", ke);
326 }
327 reconnectPeer(ke, peer);
328 addresses = Collections.emptyList();
329 }
330 peer.setRegionServers(addresses);
331 }
332
333 return peer.getRegionServers();
334 }
335
336 @Override
337 public UUID getPeerUUID(String peerId) {
338 ReplicationPeer peer = this.peerClusters.get(peerId);
339 if (peer == null) {
340 return null;
341 }
342 UUID peerUUID = null;
343
344
345
346
347 synchronized (peer) {
348 try {
349 peerUUID = ZKClusterId.getUUIDForCluster(peer.getZkw());
350 } catch (KeeperException ke) {
351 reconnectPeer(ke, peer);
352 }
353 }
354 return peerUUID;
355 }
356
357 @Override
358 public Set<String> getConnectedPeers() {
359 return this.peerClusters.keySet();
360 }
361
362 @Override
363 public Configuration getPeerConf(String peerId) throws ReplicationException {
364 String znode = ZKUtil.joinZNode(this.peersZNode, peerId);
365 byte[] data = null;
366 try {
367 data = ZKUtil.getData(this.zookeeper, znode);
368 } catch (KeeperException e) {
369 throw new ReplicationException("Error getting configuration for peer with id="
370 + peerId, e);
371 }
372 if (data == null) {
373 LOG.error("Could not get configuration for peer because it doesn't exist. peerId=" + peerId);
374 return null;
375 }
376 String otherClusterKey = "";
377 try {
378 otherClusterKey = parsePeerFrom(data);
379 } catch (DeserializationException e) {
380 LOG.warn("Failed to parse cluster key from peerId=" + peerId
381 + ", specifically the content from the following znode: " + znode);
382 return null;
383 }
384
385 Configuration otherConf = new Configuration(this.conf);
386 try {
387 ZKUtil.applyClusterKeyToConf(otherConf, otherClusterKey);
388 } catch (IOException e) {
389 LOG.error("Can't get peer configuration for peerId=" + peerId + " because:", e);
390 return null;
391 }
392 return otherConf;
393 }
394
395
396
397
398 @Override
399 public List<String> getAllPeerIds() {
400 List<String> ids = null;
401 try {
402 ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode);
403 } catch (KeeperException e) {
404 this.abortable.abort("Cannot get the list of peers ", e);
405 }
406 return ids;
407 }
408
409 @Override
410 public long getTimestampOfLastChangeToPeer(String peerId) {
411 ReplicationPeer peer = this.peerClusters.get(peerId);
412 if (peer == null) {
413 throw new IllegalArgumentException("Unknown peer id: " + peerId);
414 }
415 return peer.getLastRegionserverUpdate();
416 }
417
418
419
420
421
422 private void connectExistingPeers() throws ReplicationException {
423 List<String> znodes = null;
424 try {
425 znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
426 } catch (KeeperException e) {
427 throw new ReplicationException("Error getting the list of peer clusters.", e);
428 }
429 if (znodes != null) {
430 for (String z : znodes) {
431 connectToPeer(z);
432 }
433 }
434 }
435
436
437
438
439
440
441 private void reconnectPeer(KeeperException ke, ReplicationPeer peer) {
442 if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException
443 || ke instanceof AuthFailedException) {
444 LOG.warn("Lost the ZooKeeper connection for peer " + peer.getClusterKey(), ke);
445 try {
446 peer.reloadZkWatcher();
447 peer.getZkw().registerListener(new PeerRegionServerListener(peer));
448 } catch (IOException io) {
449 LOG.warn("Creation of ZookeeperWatcher failed for peer " + peer.getClusterKey(), io);
450 }
451 }
452 }
453
454
455
456
457
458
459 private static List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
460 throws KeeperException {
461 List<String> children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.rsZNode);
462 if (children == null) {
463 return Collections.emptyList();
464 }
465 List<ServerName> addresses = new ArrayList<ServerName>(children.size());
466 for (String child : children) {
467 addresses.add(ServerName.parseServerName(child));
468 }
469 return addresses;
470 }
471
472 private String getTableCFsNode(String id) {
473 return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
474 }
475
476 private String getPeerStateNode(String id) {
477 return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
478 }
479
480
481
482
483
484
485 private void changePeerState(String id, ZooKeeperProtos.ReplicationState.State state)
486 throws ReplicationException {
487 try {
488 if (!peerExists(id)) {
489 throw new IllegalArgumentException("Cannot enable/disable peer because id=" + id
490 + " does not exist.");
491 }
492 String peerStateZNode = getPeerStateNode(id);
493 byte[] stateBytes =
494 (state == ZooKeeperProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
495 : DISABLED_ZNODE_BYTES;
496 if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
497 ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes);
498 } else {
499 ZKUtil.createAndWatch(this.zookeeper, peerStateZNode, stateBytes);
500 }
501 LOG.info("Peer with id= " + id + " is now " + state.name());
502 } catch (KeeperException e) {
503 throw new ReplicationException("Unable to change state of the peer with id=" + id, e);
504 }
505 }
506
507
508
509
510
511
512
513 private ReplicationPeer getPeer(String peerId) throws ReplicationException {
514 Configuration peerConf = getPeerConf(peerId);
515 if (peerConf == null) {
516 return null;
517 }
518 if (this.ourClusterKey.equals(ZKUtil.getZooKeeperClusterKey(peerConf))) {
519 LOG.debug("Not connecting to " + peerId + " because it's us");
520 return null;
521 }
522
523 ReplicationPeer peer =
524 new ReplicationPeer(peerConf, peerId);
525 try {
526 peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId));
527 } catch (KeeperException e) {
528 throw new ReplicationException("Error starting the peer state tracker for peerId=" +
529 peerId, e);
530 }
531
532 try {
533 peer.startTableCFsTracker(this.zookeeper, this.getTableCFsNode(peerId));
534 } catch (KeeperException e) {
535 throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
536 peerId, e);
537 }
538
539 peer.getZkw().registerListener(new PeerRegionServerListener(peer));
540 return peer;
541 }
542
543
544
545
546
547
548 private static String parsePeerFrom(final byte[] bytes) throws DeserializationException {
549 if (ProtobufUtil.isPBMagicPrefix(bytes)) {
550 int pblen = ProtobufUtil.lengthOfPBMagic();
551 ZooKeeperProtos.ReplicationPeer.Builder builder =
552 ZooKeeperProtos.ReplicationPeer.newBuilder();
553 ZooKeeperProtos.ReplicationPeer peer;
554 try {
555 peer = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
556 } catch (InvalidProtocolBufferException e) {
557 throw new DeserializationException(e);
558 }
559 return peer.getClusterkey();
560 } else {
561 if (bytes.length > 0) {
562 return Bytes.toString(bytes);
563 }
564 return "";
565 }
566 }
567
568
569
570
571
572
573
574 private static byte[] toByteArray(final String clusterKey) {
575 byte[] bytes =
576 ZooKeeperProtos.ReplicationPeer.newBuilder().setClusterkey(clusterKey).build()
577 .toByteArray();
578 return ProtobufUtil.prependPBMagic(bytes);
579 }
580
581
582
583
584 public static class PeerRegionServerListener extends ZooKeeperListener {
585
586 private ReplicationPeer peer;
587 private String regionServerListNode;
588
589 public PeerRegionServerListener(ReplicationPeer replicationPeer) {
590 super(replicationPeer.getZkw());
591 this.peer = replicationPeer;
592 this.regionServerListNode = peer.getZkw().rsZNode;
593 }
594
595 public PeerRegionServerListener(String regionServerListNode, ZooKeeperWatcher zkw) {
596 super(zkw);
597 this.regionServerListNode = regionServerListNode;
598 }
599
600 @Override
601 public synchronized void nodeChildrenChanged(String path) {
602 if (path.equals(regionServerListNode)) {
603 try {
604 LOG.info("Detected change to peer regionservers, fetching updated list");
605 peer.setRegionServers(fetchSlavesAddresses(peer.getZkw()));
606 } catch (KeeperException e) {
607 LOG.fatal("Error reading slave addresses", e);
608 }
609 }
610 }
611
612 }
613 }