1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.replication.regionserver;
22
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.SortedMap;
28 import java.util.SortedSet;
29 import java.util.TreeSet;
30 import java.util.concurrent.atomic.AtomicBoolean;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.FileSystem;
36 import org.apache.hadoop.fs.Path;
37 import org.apache.hadoop.hbase.Stoppable;
38 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
39 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
40 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
41 import org.apache.zookeeper.KeeperException;
42
43
44
45
46
47
48
49
50
51
52
53
54 public class ReplicationSourceManager {
55 private static final Log LOG =
56 LogFactory.getLog(ReplicationSourceManager.class);
57
58 private final List<ReplicationSourceInterface> sources;
59
60 private final List<ReplicationSourceInterface> oldsources;
61
62 private final AtomicBoolean replicating;
63
64 private final ReplicationZookeeper zkHelper;
65
66 private final Stoppable stopper;
67
68 private final SortedSet<String> hlogs;
69 private final Configuration conf;
70 private final FileSystem fs;
71
72 private Path latestPath;
73
74 private final List<String> otherRegionServers;
75
76 private final Path logDir;
77
78 private final Path oldLogDir;
79
80
81
82
83
84
85
86
87
88
89
90
91 public ReplicationSourceManager(final ReplicationZookeeper zkHelper,
92 final Configuration conf,
93 final Stoppable stopper,
94 final FileSystem fs,
95 final AtomicBoolean replicating,
96 final Path logDir,
97 final Path oldLogDir) {
98 this.sources = new ArrayList<ReplicationSourceInterface>();
99 this.replicating = replicating;
100 this.zkHelper = zkHelper;
101 this.stopper = stopper;
102 this.hlogs = new TreeSet<String>();
103 this.oldsources = new ArrayList<ReplicationSourceInterface>();
104 this.conf = conf;
105 this.fs = fs;
106 this.logDir = logDir;
107 this.oldLogDir = oldLogDir;
108 this.zkHelper.registerRegionServerListener(
109 new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
110 List<String> otherRSs =
111 this.zkHelper.getRegisteredRegionServers();
112 this.zkHelper.registerRegionServerListener(
113 new PeersWatcher(this.zkHelper.getZookeeperWatcher()));
114 this.zkHelper.listPeersIdsAndWatch();
115 this.otherRegionServers = otherRSs == null ? new ArrayList<String>() : otherRSs;
116 }
117
118
119
120
121
122
123
124
125
126
127
128 public void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered) {
129 String key = log.getName();
130 LOG.info("Going to report log #" + key + " for position " + position + " in " + log);
131 this.zkHelper.writeReplicationStatus(key.toString(), id, position);
132 synchronized (this.hlogs) {
133 if (!queueRecovered && this.hlogs.first() != key) {
134 SortedSet<String> hlogSet = this.hlogs.headSet(key);
135 LOG.info("Removing " + hlogSet.size() +
136 " logs in the list: " + hlogSet);
137 for (String hlog : hlogSet) {
138 this.zkHelper.removeLogFromList(hlog.toString(), id);
139 }
140 hlogSet.clear();
141 }
142 }
143 }
144
145
146
147
148
149 public void init() throws IOException {
150 for (String id : this.zkHelper.getPeerClusters().keySet()) {
151 addSource(id);
152 }
153 List<String> currentReplicators = this.zkHelper.getListOfReplicators();
154 if (currentReplicators == null || currentReplicators.size() == 0) {
155 return;
156 }
157 synchronized (otherRegionServers) {
158 LOG.info("Current list of replicators: " + currentReplicators
159 + " other RSs: " + otherRegionServers);
160 }
161
162 for (String rs : currentReplicators) {
163 synchronized (otherRegionServers) {
164 if (!this.otherRegionServers.contains(rs)) {
165 transferQueues(rs);
166 }
167 }
168 }
169 }
170
171
172
173
174
175
176
177 public ReplicationSourceInterface addSource(String id) throws IOException {
178 ReplicationSourceInterface src =
179 getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
180
181 src.setSourceEnabled(true);
182 synchronized (this.hlogs) {
183 this.sources.add(src);
184 if (this.hlogs.size() > 0) {
185
186 this.zkHelper.addLogToList(this.hlogs.last(),
187 this.sources.get(0).getPeerClusterZnode());
188 src.enqueueLog(this.latestPath);
189 }
190 }
191 src.startup();
192 return src;
193 }
194
195
196
197
198 public void join() {
199 if (this.sources.size() == 0) {
200 this.zkHelper.deleteOwnRSZNode();
201 }
202 for (ReplicationSourceInterface source : this.sources) {
203 source.terminate("Region server is closing");
204 }
205 }
206
207
208
209
210
211 protected SortedSet<String> getHLogs() {
212 return new TreeSet<String>(this.hlogs);
213 }
214
215
216
217
218
219 public List<ReplicationSourceInterface> getSources() {
220 return this.sources;
221 }
222
223 void logRolled(Path newLog) {
224 if (!this.replicating.get()) {
225 LOG.warn("Replication stopped, won't add new log");
226 return;
227 }
228
229 if (this.sources.size() > 0) {
230 this.zkHelper.addLogToList(newLog.getName(),
231 this.sources.get(0).getPeerClusterZnode());
232 }
233 synchronized (this.hlogs) {
234 this.hlogs.add(newLog.getName());
235 }
236 this.latestPath = newLog;
237
238 for (ReplicationSourceInterface source : this.sources) {
239 source.enqueueLog(newLog);
240 }
241 }
242
243
244
245
246
247 public ReplicationZookeeper getRepZkWrapper() {
248 return zkHelper;
249 }
250
251
252
253
254
255
256
257
258
259
260
261
262 public ReplicationSourceInterface getReplicationSource(
263 final Configuration conf,
264 final FileSystem fs,
265 final ReplicationSourceManager manager,
266 final Stoppable stopper,
267 final AtomicBoolean replicating,
268 final String peerClusterId) throws IOException {
269 ReplicationSourceInterface src;
270 try {
271 @SuppressWarnings("rawtypes")
272 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
273 ReplicationSource.class.getCanonicalName()));
274 src = (ReplicationSourceInterface) c.newInstance();
275 } catch (Exception e) {
276 LOG.warn("Passed replication source implemention throws errors, " +
277 "defaulting to ReplicationSource", e);
278 src = new ReplicationSource();
279
280 }
281 src.init(conf, fs, manager, stopper, replicating, peerClusterId);
282 return src;
283 }
284
285
286
287
288
289
290
291
292
293 public void transferQueues(String rsZnode) {
294
295 if (this.stopper.isStopped()) {
296 LOG.info("Not transferring queue since we are shutting down");
297 return;
298 }
299 if (!this.zkHelper.lockOtherRS(rsZnode)) {
300 return;
301 }
302 LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
303 SortedMap<String, SortedSet<String>> newQueues =
304 this.zkHelper.copyQueuesFromRS(rsZnode);
305 this.zkHelper.deleteRsQueues(rsZnode);
306 if (newQueues == null || newQueues.size() == 0) {
307 return;
308 }
309
310 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
311 String peerId = entry.getKey();
312 try {
313 ReplicationSourceInterface src = getReplicationSource(this.conf,
314 this.fs, this, this.stopper, this.replicating, peerId);
315 if (!zkHelper.getPeerClusters().containsKey(src.getPeerClusterId())) {
316 src.terminate("Recovered queue doesn't belong to any current peer");
317 break;
318 }
319 this.oldsources.add(src);
320 for (String hlog : entry.getValue()) {
321 src.enqueueLog(new Path(this.oldLogDir, hlog));
322 }
323
324 src.setSourceEnabled(true);
325 src.startup();
326 } catch (IOException e) {
327
328 LOG.error("Failed creating a source", e);
329 }
330 }
331 }
332
333
334
335
336
337 public void closeRecoveredQueue(ReplicationSourceInterface src) {
338 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
339 this.oldsources.remove(src);
340 this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
341 }
342
343
344
345
346
347
348 public void removePeer(String id) {
349 LOG.info("Closing the following queue " + id + ", currently have "
350 + sources.size() + " and another "
351 + oldsources.size() + " that were recovered");
352 ReplicationSourceInterface srcToRemove = null;
353 List<ReplicationSourceInterface> oldSourcesToDelete =
354 new ArrayList<ReplicationSourceInterface>();
355
356 for (ReplicationSourceInterface src : oldsources) {
357 if (id.equals(src.getPeerClusterId())) {
358 oldSourcesToDelete.add(src);
359 }
360 }
361 for (ReplicationSourceInterface src : oldSourcesToDelete) {
362 closeRecoveredQueue((src));
363 }
364 LOG.info("Number of deleted recovered sources for " + id + ": "
365 + oldSourcesToDelete.size());
366
367 for (ReplicationSourceInterface src : this.sources) {
368 if (id.equals(src.getPeerClusterId())) {
369 srcToRemove = src;
370 break;
371 }
372 }
373 if (srcToRemove == null) {
374 LOG.error("The queue we wanted to close is missing " + id);
375 return;
376 }
377 srcToRemove.terminate("Replication stream was removed by a user");
378 this.sources.remove(srcToRemove);
379 this.zkHelper.deleteSource(id, true);
380 }
381
382
383
384
385
386
387 public class OtherRegionServerWatcher extends ZooKeeperListener {
388
389
390
391
392 public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
393 super(watcher);
394 }
395
396
397
398
399
400 public void nodeCreated(String path) {
401 refreshRegionServersList(path);
402 }
403
404
405
406
407
408 public void nodeDeleted(String path) {
409 if (stopper.isStopped()) {
410 return;
411 }
412 boolean cont = refreshRegionServersList(path);
413 if (!cont) {
414 return;
415 }
416 LOG.info(path + " znode expired, trying to lock it");
417 transferQueues(zkHelper.getZNodeName(path));
418 }
419
420
421
422
423
424 public void nodeChildrenChanged(String path) {
425 if (stopper.isStopped()) {
426 return;
427 }
428 refreshRegionServersList(path);
429 }
430
431 private boolean refreshRegionServersList(String path) {
432 if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
433 return false;
434 }
435 List<String> newRsList = (zkHelper.getRegisteredRegionServers());
436 if (newRsList == null) {
437 return false;
438 } else {
439 synchronized (otherRegionServers) {
440 otherRegionServers.clear();
441 otherRegionServers.addAll(newRsList);
442 }
443 }
444 return true;
445 }
446 }
447
448
449
450
451 public class PeersWatcher extends ZooKeeperListener {
452
453
454
455
456 public PeersWatcher(ZooKeeperWatcher watcher) {
457 super(watcher);
458 }
459
460
461
462
463
464 public void nodeDeleted(String path) {
465 List<String> peers = refreshPeersList(path);
466 if (peers == null) {
467 return;
468 }
469 String id = zkHelper.getZNodeName(path);
470 removePeer(id);
471 }
472
473
474
475
476
477 public void nodeChildrenChanged(String path) {
478 List<String> peers = refreshPeersList(path);
479 if (peers == null) {
480 return;
481 }
482 for (String id : peers) {
483 try {
484 boolean added = zkHelper.connectToPeer(id);
485 if (added) {
486 addSource(id);
487 }
488 } catch (IOException e) {
489
490 LOG.error("Error while adding a new peer", e);
491 } catch (KeeperException e) {
492 LOG.error("Error while adding a new peer", e);
493 }
494 }
495 }
496
497
498
499
500
501
502
503
504 private List<String> refreshPeersList(String path) {
505 if (!path.startsWith(zkHelper.getPeersZNode())) {
506 return null;
507 }
508 return zkHelper.listPeersIdsAndWatch();
509 }
510 }
511
512
513
514
515
516 public Path getOldLogDir() {
517 return this.oldLogDir;
518 }
519
520
521
522
523
524 public Path getLogDir() {
525 return this.logDir;
526 }
527
528
529
530
531
532 public FileSystem getFs() {
533 return this.fs;
534 }
535 }