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