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