1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.replication;
21
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.hbase.HConstants;
26 import org.apache.hadoop.hbase.HServerAddress;
27 import org.apache.hadoop.hbase.util.Bytes;
28 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
29 import org.apache.hadoop.util.StringUtils;
30 import org.apache.zookeeper.KeeperException;
31 import org.apache.zookeeper.WatchedEvent;
32 import org.apache.zookeeper.Watcher;
33
34 import java.io.IOException;
35 import java.util.ArrayList;
36 import java.util.HashMap;
37 import java.util.List;
38 import java.util.Map;
39 import java.util.SortedMap;
40 import java.util.TreeMap;
41 import java.util.SortedSet;
42 import java.util.TreeSet;
43 import java.util.concurrent.atomic.AtomicBoolean;
44
45
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 public class ReplicationZookeeperWrapper {
74
75 private static final Log LOG =
76 LogFactory.getLog(ReplicationZookeeperWrapper.class);
77
78 private final static String RS_LOCK_ZNODE = "lock";
79
80 private final ZooKeeperWrapper zookeeperWrapper;
81
82 private final Map<String, ZooKeeperWrapper> peerClusters;
83
84 private final String replicationZNode;
85
86 private final String peersZNode;
87
88 private final String rsZNode;
89
90 private final String rsServerNameZnode;
91
92 private final String replicationStateNodeName;
93
94 private final boolean replicationMaster;
95 private final Configuration conf;
96
97 private final AtomicBoolean replicating;
98
99 private final String clusterId;
100
101
102
103
104
105
106
107
108
109
110
111 public ReplicationZookeeperWrapper(
112 ZooKeeperWrapper zookeeperWrapper, Configuration conf,
113 final AtomicBoolean replicating, String rsName) throws IOException {
114 this.zookeeperWrapper = zookeeperWrapper;
115 this.conf = conf;
116 String replicationZNodeName =
117 conf.get("zookeeper.znode.replication", "replication");
118 String peersZNodeName =
119 conf.get("zookeeper.znode.replication.peers", "peers");
120 String repMasterZNodeName =
121 conf.get("zookeeper.znode.replication.master", "master");
122 this.replicationStateNodeName =
123 conf.get("zookeeper.znode.replication.state", "state");
124 String clusterIdZNodeName =
125 conf.get("zookeeper.znode.replication.clusterId", "clusterId");
126 String rsZNodeName =
127 conf.get("zookeeper.znode.replication.rs", "rs");
128 String thisCluster = this.conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" +
129 this.conf.get("hbase.zookeeper.property.clientPort") + ":" +
130 this.conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
131
132 this.peerClusters = new HashMap<String, ZooKeeperWrapper>();
133 this.replicationZNode = zookeeperWrapper.getZNode(
134 zookeeperWrapper.getParentZNode(), replicationZNodeName);
135 this.peersZNode =
136 zookeeperWrapper.getZNode(replicationZNode, peersZNodeName);
137 this.rsZNode =
138 zookeeperWrapper.getZNode(replicationZNode, rsZNodeName);
139
140 this.replicating = replicating;
141 setReplicating();
142 String idResult = Bytes.toString(
143 this.zookeeperWrapper.getData(this.replicationZNode,
144 clusterIdZNodeName));
145 this.clusterId =
146 idResult == null ?
147 Byte.toString(HConstants.DEFAULT_CLUSTER_ID) : idResult;
148 String address = Bytes.toString(
149 this.zookeeperWrapper.getData(this.replicationZNode,
150 repMasterZNodeName));
151 this.replicationMaster = thisCluster.equals(address);
152 LOG.info("This cluster (" + thisCluster + ") is a "
153 + (this.replicationMaster ? "master" : "slave") + " for replication" +
154 ", compared with (" + address + ")");
155 if (rsName != null) {
156 this.rsServerNameZnode =
157 this.zookeeperWrapper.getZNode(rsZNode, rsName);
158 List<String> znodes = this.zookeeperWrapper.listZnodes(this.peersZNode,
159 new ReplicationStatusWatcher());
160 if (znodes != null) {
161 for (String znode : znodes) {
162 connectToPeer(znode);
163 }
164 }
165 } else {
166 this.rsServerNameZnode = null;
167 }
168
169 }
170
171
172
173
174
175
176
177 public List<HServerAddress> getPeersAddresses(String peerClusterId) {
178 if (this.peerClusters.size() == 0) {
179 return new ArrayList<HServerAddress>(0);
180 }
181 ZooKeeperWrapper zkw = this.peerClusters.get(peerClusterId);
182 return zkw == null?
183 new ArrayList<HServerAddress>(0) : zkw.scanRSDirectory();
184 }
185
186
187
188
189
190
191 private void connectToPeer(String peerId) throws IOException {
192 String[] ensemble =
193 Bytes.toString(this.zookeeperWrapper.getData(this.peersZNode, peerId)).
194 split(":");
195 if (ensemble.length != 3) {
196 throw new IllegalArgumentException("Wrong format of cluster address: " +
197 this.zookeeperWrapper.getData(this.peersZNode, peerId));
198 }
199 Configuration otherConf = new Configuration(this.conf);
200 otherConf.set(HConstants.ZOOKEEPER_QUORUM, ensemble[0]);
201 otherConf.set("hbase.zookeeper.property.clientPort", ensemble[1]);
202 otherConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, ensemble[2]);
203 ZooKeeperWrapper zkw = ZooKeeperWrapper.createInstance(otherConf,
204 "connection to cluster: " + peerId);
205 zkw.registerListener(new ReplicationStatusWatcher());
206 this.peerClusters.put(peerId, zkw);
207 this.zookeeperWrapper.ensureExists(this.zookeeperWrapper.getZNode(
208 this.rsServerNameZnode, peerId));
209 LOG.info("Added new peer cluster " + StringUtils.arrayToString(ensemble));
210 }
211
212
213
214
215 private void setReplicating() {
216 String value = Bytes.toString(this.zookeeperWrapper.getDataAndWatch(
217 this.replicationZNode, this.replicationStateNodeName,
218 new ReplicationStatusWatcher()));
219 if (value != null) {
220 this.replicating.set(value.equals("true"));
221 LOG.info("Replication is now " + (this.replicating.get() ?
222 "started" : "stopped"));
223 }
224 }
225
226
227
228
229
230
231 public void addLogToList(String filename, String clusterId) {
232 try {
233 this.zookeeperWrapper.writeZNode(
234 this.zookeeperWrapper.getZNode(
235 this.rsServerNameZnode, clusterId), filename, "");
236 } catch (InterruptedException e) {
237 LOG.error(e);
238 } catch (KeeperException e) {
239 LOG.error(e);
240 }
241 }
242
243
244
245
246
247
248 public void removeLogFromList(String filename, String clusterId) {
249 try {
250 this.zookeeperWrapper.deleteZNode(
251 this.zookeeperWrapper.getZNode(this.rsServerNameZnode,
252 this.zookeeperWrapper.getZNode(clusterId, filename)));
253 } catch (InterruptedException e) {
254 LOG.error(e);
255 } catch (KeeperException e) {
256 LOG.error(e);
257 }
258 }
259
260
261
262
263
264
265
266
267 public void writeReplicationStatus(String filename, String clusterId,
268 long position) {
269 try {
270 String clusterZNode = this.zookeeperWrapper.getZNode(
271 this.rsServerNameZnode, clusterId);
272 this.zookeeperWrapper.writeZNode(clusterZNode, filename,
273 Long.toString(position));
274 } catch (InterruptedException e) {
275 LOG.error(e);
276 } catch (KeeperException e) {
277 LOG.error(e);
278 }
279 }
280
281
282
283
284
285
286
287 public List<String> getRegisteredRegionServers(Watcher watch) {
288 return this.zookeeperWrapper.listZnodes(
289 this.zookeeperWrapper.getRsZNode(), watch);
290 }
291
292
293
294
295
296
297
298 public List<String> getListOfReplicators(Watcher watch) {
299 return this.zookeeperWrapper.listZnodes(rsZNode, watch);
300 }
301
302
303
304
305
306
307
308 public List<String> getListPeersForRS(String rs, Watcher watch) {
309 return this.zookeeperWrapper.listZnodes(
310 zookeeperWrapper.getZNode(rsZNode, rs), watch);
311 }
312
313
314
315
316
317
318
319
320 public List<String> getListHLogsForPeerForRS(String rs, String id, Watcher watch) {
321 return this.zookeeperWrapper.listZnodes(
322 zookeeperWrapper.getZNode(zookeeperWrapper.getZNode(rsZNode, rs), id), watch);
323 }
324
325
326
327
328
329
330 public boolean lockOtherRS(String znode) {
331 try {
332 this.zookeeperWrapper.writeZNode(
333 this.zookeeperWrapper.getZNode(this.rsZNode, znode),
334 RS_LOCK_ZNODE, rsServerNameZnode, true);
335
336 } catch (InterruptedException e) {
337 LOG.error(e);
338 return false;
339 } catch (KeeperException e) {
340 LOG.debug("Won't lock " + znode + " because " + e.getMessage());
341
342 return false;
343 }
344 return true;
345 }
346
347
348
349
350
351
352
353
354 public SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
355
356
357 SortedMap<String,SortedSet<String>> queues =
358 new TreeMap<String,SortedSet<String>>();
359 try {
360 String nodePath = this.zookeeperWrapper.getZNode(rsZNode, znode);
361 List<String> clusters = this.zookeeperWrapper.listZnodes(nodePath, null);
362
363 if (clusters == null || clusters.size() <= 1) {
364 return queues;
365 }
366
367 clusters.remove(RS_LOCK_ZNODE);
368 for (String cluster : clusters) {
369
370
371
372 String newCluster = cluster+"-"+znode;
373 String newClusterZnode =
374 this.zookeeperWrapper.getZNode(rsServerNameZnode, newCluster);
375 this.zookeeperWrapper.ensureExists(newClusterZnode);
376 String clusterPath = this.zookeeperWrapper.getZNode(nodePath, cluster);
377 List<String> hlogs = this.zookeeperWrapper.listZnodes(clusterPath, null);
378
379 if (hlogs == null || hlogs.size() == 0) {
380 continue;
381 }
382 SortedSet<String> logQueue = new TreeSet<String>();
383 queues.put(newCluster, logQueue);
384 for (String hlog : hlogs) {
385 String position = Bytes.toString(
386 this.zookeeperWrapper.getData(clusterPath, hlog));
387 LOG.debug("Creating " + hlog + " with data " + position);
388 this.zookeeperWrapper.writeZNode(newClusterZnode, hlog, position);
389 logQueue.add(hlog);
390 }
391 }
392 } catch (InterruptedException e) {
393 LOG.warn(e);
394 return null;
395 } catch (KeeperException e) {
396 LOG.warn(e);
397 return null;
398 }
399 return queues;
400 }
401
402
403
404
405
406 public void deleteSource(String peerZnode) {
407 try {
408 this.zookeeperWrapper.deleteZNode(
409 this.zookeeperWrapper.getZNode(rsServerNameZnode, peerZnode), true);
410 } catch (InterruptedException e) {
411 LOG.error(e);
412 } catch (KeeperException e) {
413 LOG.error(e);
414 }
415 }
416
417
418
419
420
421 public void deleteRsQueues(String znode) {
422 try {
423 this.zookeeperWrapper.deleteZNode(
424 this.zookeeperWrapper.getZNode(rsZNode, znode), true);
425 } catch (InterruptedException e) {
426 LOG.error(e);
427 } catch (KeeperException e) {
428 LOG.error(e);
429 }
430 }
431
432
433
434
435 public void deleteOwnRSZNode() {
436 deleteRsQueues(this.rsServerNameZnode);
437 }
438
439
440
441
442
443
444
445 public long getHLogRepPosition(String peerId, String hlog) {
446 String clusterZnode =
447 this.zookeeperWrapper.getZNode(rsServerNameZnode, peerId);
448 String data = Bytes.toString(
449 this.zookeeperWrapper.getData(clusterZnode, hlog));
450 return data == null || data.length() == 0 ? 0 : Long.parseLong(data);
451 }
452
453
454
455
456
457
458 public boolean isReplicationMaster() {
459 return this.replicationMaster;
460 }
461
462
463
464
465
466
467 public String getClusterId() {
468 return this.clusterId;
469 }
470
471
472
473
474
475 public Map<String, ZooKeeperWrapper> getPeerClusters() {
476 return this.peerClusters;
477 }
478
479
480
481
482 public class ReplicationStatusWatcher implements Watcher {
483 @Override
484 public void process(WatchedEvent watchedEvent) {
485 Event.EventType type = watchedEvent.getType();
486 LOG.info("Got event " + type + " with path " + watchedEvent.getPath());
487 if (type.equals(Event.EventType.NodeDataChanged)) {
488 setReplicating();
489 }
490 }
491 }
492
493 }