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