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.List;
24 import java.util.Map;
25 import java.util.Set;
26 import java.util.TreeMap;
27 import java.util.concurrent.ConcurrentHashMap;
28 import java.util.concurrent.ConcurrentMap;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.Abortable;
34 import org.apache.hadoop.hbase.CompoundConfiguration;
35 import org.apache.hadoop.hbase.classification.InterfaceAudience;
36 import org.apache.hadoop.hbase.exceptions.DeserializationException;
37 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
38 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
39 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
40 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
41 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.hbase.util.Pair;
44 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
45 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
46 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
47 import org.apache.zookeeper.KeeperException;
48
49 import com.google.protobuf.ByteString;
50
51
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 @InterfaceAudience.Private
78 public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements ReplicationPeers {
79
80
81 private Map<String, ReplicationPeerZKImpl> peerClusters;
82 private final String tableCFsNodeName;
83
84 private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class);
85
86 public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf,
87 Abortable abortable) {
88 super(zk, conf, abortable);
89 this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
90 this.peerClusters = new ConcurrentHashMap<String, ReplicationPeerZKImpl>();
91 }
92
93 @Override
94 public void init() throws ReplicationException {
95 try {
96 if (ZKUtil.checkExists(this.zookeeper, this.peersZNode) < 0) {
97 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
98 }
99 } catch (KeeperException e) {
100 throw new ReplicationException("Could not initialize replication peers", e);
101 }
102 addExistingPeers();
103 }
104
105 @Override
106 public void addPeer(String id, ReplicationPeerConfig peerConfig, String tableCFs)
107 throws ReplicationException {
108 try {
109 if (peerExists(id)) {
110 throw new IllegalArgumentException("Cannot add a peer with id=" + id
111 + " because that id already exists.");
112 }
113
114 if(id.contains("-")){
115 throw new IllegalArgumentException("Found invalid peer name:" + id);
116 }
117
118 ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
119 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
120 ZKUtilOp op1 =
121 ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
122 toByteArray(peerConfig));
123
124
125
126
127 ZKUtilOp op2 = ZKUtilOp.createAndFailSilent(getPeerStateNode(id), ENABLED_ZNODE_BYTES);
128 String tableCFsStr = (tableCFs == null) ? "" : tableCFs;
129 ZKUtilOp op3 = ZKUtilOp.createAndFailSilent(getTableCFsNode(id), Bytes.toBytes(tableCFsStr));
130 listOfOps.add(op1);
131 listOfOps.add(op2);
132 listOfOps.add(op3);
133 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
134 } catch (KeeperException e) {
135 throw new ReplicationException("Could not add peer with id=" + id + ", peerConfif="
136 + peerConfig, e);
137 }
138 }
139
140 @Override
141 public void removePeer(String id) throws ReplicationException {
142 try {
143 if (!peerExists(id)) {
144 throw new IllegalArgumentException("Cannot remove peer with id=" + id
145 + " because that id does not exist.");
146 }
147 ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
148 } catch (KeeperException e) {
149 throw new ReplicationException("Could not remove peer with id=" + id, e);
150 }
151 }
152
153 @Override
154 public void enablePeer(String id) throws ReplicationException {
155 changePeerState(id, ZooKeeperProtos.ReplicationState.State.ENABLED);
156 LOG.info("peer " + id + " is enabled");
157 }
158
159 @Override
160 public void disablePeer(String id) throws ReplicationException {
161 changePeerState(id, ZooKeeperProtos.ReplicationState.State.DISABLED);
162 LOG.info("peer " + id + " is disabled");
163 }
164
165 @Override
166 public String getPeerTableCFsConfig(String id) throws ReplicationException {
167 try {
168 if (!peerExists(id)) {
169 throw new IllegalArgumentException("peer " + id + " doesn't exist");
170 }
171 try {
172 return Bytes.toString(ZKUtil.getData(this.zookeeper, getTableCFsNode(id)));
173 } catch (Exception e) {
174 throw new ReplicationException(e);
175 }
176 } catch (KeeperException e) {
177 throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id, e);
178 }
179 }
180
181 @Override
182 public void setPeerTableCFsConfig(String id, String tableCFsStr) throws ReplicationException {
183 try {
184 if (!peerExists(id)) {
185 throw new IllegalArgumentException("Cannot set peer tableCFs because id=" + id
186 + " does not exist.");
187 }
188 String tableCFsZKNode = getTableCFsNode(id);
189 byte[] tableCFs = Bytes.toBytes(tableCFsStr);
190 if (ZKUtil.checkExists(this.zookeeper, tableCFsZKNode) != -1) {
191 ZKUtil.setData(this.zookeeper, tableCFsZKNode, tableCFs);
192 } else {
193 ZKUtil.createAndWatch(this.zookeeper, tableCFsZKNode, tableCFs);
194 }
195 LOG.info("Peer tableCFs with id= " + id + " is now " + tableCFsStr);
196 } catch (KeeperException e) {
197 throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e);
198 }
199 }
200
201 @Override
202 public Map<String, List<String>> getTableCFs(String id) throws IllegalArgumentException {
203 ReplicationPeer replicationPeer = this.peerClusters.get(id);
204 if (replicationPeer == null) {
205 throw new IllegalArgumentException("Peer with id= " + id + " is not connected");
206 }
207 return replicationPeer.getTableCFs();
208 }
209
210 @Override
211 public boolean getStatusOfPeer(String id) {
212 ReplicationPeer replicationPeer = this.peerClusters.get(id);
213 if (replicationPeer == null) {
214 throw new IllegalArgumentException("Peer with id= " + id + " is not connected");
215 }
216 return this.peerClusters.get(id).getPeerState() == PeerState.ENABLED;
217 }
218
219 @Override
220 public boolean getStatusOfPeerFromBackingStore(String id) throws ReplicationException {
221 try {
222 if (!peerExists(id)) {
223 throw new IllegalArgumentException("peer " + id + " doesn't exist");
224 }
225 String peerStateZNode = getPeerStateNode(id);
226 try {
227 return ReplicationPeerZKImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode));
228 } catch (KeeperException e) {
229 throw new ReplicationException(e);
230 } catch (DeserializationException e) {
231 throw new ReplicationException(e);
232 }
233 } catch (KeeperException e) {
234 throw new ReplicationException("Unable to get status of the peer with id=" + id +
235 " from backing store", e);
236 }
237 }
238
239 @Override
240 public Map<String, ReplicationPeerConfig> getAllPeerConfigs() {
241 Map<String, ReplicationPeerConfig> peers = new TreeMap<String, ReplicationPeerConfig>();
242 List<String> ids = null;
243 try {
244 ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
245 for (String id : ids) {
246 ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id);
247 if (peerConfig == null) {
248 LOG.warn("Failed to get replication peer configuration of clusterid=" + id
249 + " znode content, continuing.");
250 continue;
251 }
252 peers.put(id, peerConfig);
253 }
254 } catch (KeeperException e) {
255 this.abortable.abort("Cannot get the list of peers ", e);
256 } catch (ReplicationException e) {
257 this.abortable.abort("Cannot get the list of peers ", e);
258 }
259 return peers;
260 }
261
262 @Override
263 public ReplicationPeerConfig getReplicationPeerConfig(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=" + peerId, e);
270 }
271 if (data == null) {
272 LOG.error("Could not get configuration for peer because it doesn't exist. peerId=" + peerId);
273 return null;
274 }
275
276 try {
277 return parsePeerFrom(data);
278 } catch (DeserializationException e) {
279 LOG.warn("Failed to parse cluster key from peerId=" + peerId
280 + ", specifically the content from the following znode: " + znode);
281 return null;
282 }
283 }
284
285 @Override
286 public Pair<ReplicationPeerConfig, Configuration> getPeerConf(String peerId)
287 throws ReplicationException {
288 ReplicationPeerConfig peerConfig = getReplicationPeerConfig(peerId);
289
290 if (peerConfig == null) {
291 return null;
292 }
293
294 Configuration otherConf = new Configuration(this.conf);
295 try {
296 if (peerConfig.getClusterKey() != null && !peerConfig.getClusterKey().isEmpty()) {
297 ZKUtil.applyClusterKeyToConf(otherConf, peerConfig.getClusterKey());
298 }
299 } catch (IOException e) {
300 LOG.error("Can't get peer configuration for peerId=" + peerId + " because:", e);
301 return null;
302 }
303
304 if (!peerConfig.getConfiguration().isEmpty()) {
305 CompoundConfiguration compound = new CompoundConfiguration();
306 compound.add(otherConf);
307 compound.addStringMap(peerConfig.getConfiguration());
308 return new Pair<ReplicationPeerConfig, Configuration>(peerConfig, compound);
309 }
310
311 return new Pair<ReplicationPeerConfig, Configuration>(peerConfig, otherConf);
312 }
313
314 @Override
315 public Set<String> getPeerIds() {
316 return peerClusters.keySet();
317 }
318
319 @Override
320 public ReplicationPeer getPeer(String peerId) {
321 return peerClusters.get(peerId);
322 }
323
324
325
326
327 @Override
328 public List<String> getAllPeerIds() {
329 List<String> ids = null;
330 try {
331 ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode);
332 } catch (KeeperException e) {
333 this.abortable.abort("Cannot get the list of peers ", e);
334 }
335 return ids;
336 }
337
338
339
340
341
342 private void addExistingPeers() throws ReplicationException {
343 List<String> znodes = null;
344 try {
345 znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
346 } catch (KeeperException e) {
347 throw new ReplicationException("Error getting the list of peer clusters.", e);
348 }
349 if (znodes != null) {
350 for (String z : znodes) {
351 createAndAddPeer(z);
352 }
353 }
354 }
355
356 @Override
357 public boolean peerAdded(String peerId) throws ReplicationException {
358 return createAndAddPeer(peerId);
359 }
360
361 @Override
362 public void peerRemoved(String peerId) {
363 ReplicationPeer rp = this.peerClusters.get(peerId);
364 if (rp != null) {
365 this.peerClusters.remove(peerId);
366 }
367 }
368
369
370
371
372
373
374 public boolean createAndAddPeer(String peerId) throws ReplicationException {
375 if (peerClusters == null) {
376 return false;
377 }
378 if (this.peerClusters.containsKey(peerId)) {
379 return false;
380 }
381
382 ReplicationPeerZKImpl peer = null;
383 try {
384 peer = createPeer(peerId);
385 } catch (Exception e) {
386 throw new ReplicationException("Error adding peer with id=" + peerId, e);
387 }
388 if (peer == null) {
389 return false;
390 }
391 ReplicationPeerZKImpl previous = ((ConcurrentMap<String, ReplicationPeerZKImpl>) peerClusters)
392 .putIfAbsent(peerId, peer);
393 if (previous == null) {
394 LOG.info("Added new peer cluster=" + peer.getPeerConfig().getClusterKey());
395 } else {
396 LOG.info("Peer already present, " + previous.getPeerConfig().getClusterKey()
397 + ", new cluster=" + peer.getPeerConfig().getClusterKey());
398 }
399 return true;
400 }
401
402 private String getTableCFsNode(String id) {
403 return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
404 }
405
406 private String getPeerStateNode(String id) {
407 return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
408 }
409
410
411
412
413
414
415 private void changePeerState(String id, ZooKeeperProtos.ReplicationState.State state)
416 throws ReplicationException {
417 try {
418 if (!peerExists(id)) {
419 throw new IllegalArgumentException("Cannot enable/disable peer because id=" + id
420 + " does not exist.");
421 }
422 String peerStateZNode = getPeerStateNode(id);
423 byte[] stateBytes =
424 (state == ZooKeeperProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
425 : DISABLED_ZNODE_BYTES;
426 if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
427 ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes);
428 } else {
429 ZKUtil.createAndWatch(this.zookeeper, peerStateZNode, stateBytes);
430 }
431 LOG.info("Peer with id= " + id + " is now " + state.name());
432 } catch (KeeperException e) {
433 throw new ReplicationException("Unable to change state of the peer with id=" + id, e);
434 }
435 }
436
437
438
439
440
441
442
443 private ReplicationPeerZKImpl createPeer(String peerId) throws ReplicationException {
444 Pair<ReplicationPeerConfig, Configuration> pair = getPeerConf(peerId);
445 if (pair == null) {
446 return null;
447 }
448 Configuration peerConf = pair.getSecond();
449
450 ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
451 try {
452 peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId));
453 } catch (KeeperException e) {
454 throw new ReplicationException("Error starting the peer state tracker for peerId=" +
455 peerId, e);
456 }
457
458 try {
459 peer.startTableCFsTracker(this.zookeeper, this.getTableCFsNode(peerId));
460 } catch (KeeperException e) {
461 throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
462 peerId, e);
463 }
464
465 return peer;
466 }
467
468
469
470
471
472
473 private static ReplicationPeerConfig parsePeerFrom(final byte[] bytes)
474 throws DeserializationException {
475 if (ProtobufUtil.isPBMagicPrefix(bytes)) {
476 int pblen = ProtobufUtil.lengthOfPBMagic();
477 ZooKeeperProtos.ReplicationPeer.Builder builder =
478 ZooKeeperProtos.ReplicationPeer.newBuilder();
479 ZooKeeperProtos.ReplicationPeer peer;
480 try {
481 ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
482 peer = builder.build();
483 } catch (IOException e) {
484 throw new DeserializationException(e);
485 }
486 return convert(peer);
487 } else {
488 if (bytes.length > 0) {
489 return new ReplicationPeerConfig().setClusterKey(Bytes.toString(bytes));
490 }
491 return new ReplicationPeerConfig().setClusterKey("");
492 }
493 }
494
495 private static ReplicationPeerConfig convert(ZooKeeperProtos.ReplicationPeer peer) {
496 ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
497 if (peer.hasClusterkey()) {
498 peerConfig.setClusterKey(peer.getClusterkey());
499 }
500 if (peer.hasReplicationEndpointImpl()) {
501 peerConfig.setReplicationEndpointImpl(peer.getReplicationEndpointImpl());
502 }
503
504 for (BytesBytesPair pair : peer.getDataList()) {
505 peerConfig.getPeerData().put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
506 }
507
508 for (NameStringPair pair : peer.getConfigurationList()) {
509 peerConfig.getConfiguration().put(pair.getName(), pair.getValue());
510 }
511 return peerConfig;
512 }
513
514 private static ZooKeeperProtos.ReplicationPeer convert(ReplicationPeerConfig peerConfig) {
515 ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
516 if (peerConfig.getClusterKey() != null) {
517 builder.setClusterkey(peerConfig.getClusterKey());
518 }
519 if (peerConfig.getReplicationEndpointImpl() != null) {
520 builder.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
521 }
522
523 for (Map.Entry<byte[], byte[]> entry : peerConfig.getPeerData().entrySet()) {
524 builder.addData(BytesBytesPair.newBuilder()
525 .setFirst(ByteString.copyFrom(entry.getKey()))
526 .setSecond(ByteString.copyFrom(entry.getValue()))
527 .build());
528 }
529
530 for (Map.Entry<String, String> entry : peerConfig.getConfiguration().entrySet()) {
531 builder.addConfiguration(NameStringPair.newBuilder()
532 .setName(entry.getKey())
533 .setValue(entry.getValue())
534 .build());
535 }
536
537 return builder.build();
538 }
539
540
541
542
543
544
545
546 private static byte[] toByteArray(final ReplicationPeerConfig peerConfig) {
547 byte[] bytes = convert(peerConfig).toByteArray();
548 return ProtobufUtil.prependPBMagic(bytes);
549 }
550
551
552 }