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