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