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