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