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.Collections;
26 import java.util.HashMap;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.Random;
30 import java.util.SortedMap;
31 import java.util.SortedSet;
32 import java.util.TreeSet;
33 import java.util.concurrent.LinkedBlockingQueue;
34 import java.util.concurrent.RejectedExecutionException;
35 import java.util.concurrent.ThreadPoolExecutor;
36 import java.util.concurrent.TimeUnit;
37 import java.util.concurrent.atomic.AtomicBoolean;
38
39 import org.apache.commons.logging.Log;
40 import org.apache.commons.logging.LogFactory;
41 import org.apache.hadoop.conf.Configuration;
42 import org.apache.hadoop.fs.FileSystem;
43 import org.apache.hadoop.fs.Path;
44 import org.apache.hadoop.hbase.HConstants;
45 import org.apache.hadoop.hbase.Stoppable;
46 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
47 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
48 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
49 import org.apache.zookeeper.KeeperException;
50
51 import com.google.common.util.concurrent.ThreadFactoryBuilder;
52
53
54
55
56
57
58
59
60
61
62
63
64 public class ReplicationSourceManager {
65 private static final Log LOG =
66 LogFactory.getLog(ReplicationSourceManager.class);
67
68 private final List<ReplicationSourceInterface> sources;
69
70 private final List<ReplicationSourceInterface> oldsources;
71
72 private final AtomicBoolean replicating;
73
74 private final ReplicationZookeeper zkHelper;
75
76 private final Stoppable stopper;
77
78 private final Map<String, SortedSet<String>> hlogsById;
79 private final Configuration conf;
80 private final FileSystem fs;
81
82 private Path latestPath;
83
84 private final List<String> otherRegionServers = new ArrayList<String>();
85
86 private final Path logDir;
87
88 private final Path oldLogDir;
89
90 private final long sleepBeforeFailover;
91
92 private final ThreadPoolExecutor executor;
93
94 private final Random rand;
95
96
97
98
99
100
101
102
103
104
105
106
107
108 public ReplicationSourceManager(final ReplicationZookeeper zkHelper,
109 final Configuration conf,
110 final Stoppable stopper,
111 final FileSystem fs,
112 final AtomicBoolean replicating,
113 final Path logDir,
114 final Path oldLogDir) {
115 this.sources = new ArrayList<ReplicationSourceInterface>();
116 this.replicating = replicating;
117 this.zkHelper = zkHelper;
118 this.stopper = stopper;
119 this.hlogsById = new HashMap<String, SortedSet<String>>();
120 this.oldsources = new ArrayList<ReplicationSourceInterface>();
121 this.conf = conf;
122 this.fs = fs;
123 this.logDir = logDir;
124 this.oldLogDir = oldLogDir;
125 this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 2000);
126 this.zkHelper.registerRegionServerListener(
127 new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
128 this.zkHelper.registerRegionServerListener(
129 new PeersWatcher(this.zkHelper.getZookeeperWatcher()));
130 this.zkHelper.listPeersIdsAndWatch();
131
132
133 int nbWorkers = conf.getInt("replication.executor.workers", 1);
134
135
136 this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers,
137 100, TimeUnit.MILLISECONDS,
138 new LinkedBlockingQueue<Runnable>());
139 ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
140 tfb.setNameFormat("ReplicationExecutor-%d");
141 tfb.setDaemon(true);
142 this.executor.setThreadFactory(tfb.build());
143 this.rand = new Random();
144 }
145
146
147
148
149
150
151
152
153
154
155
156
157 public void logPositionAndCleanOldLogs(Path log, String id, long position,
158 boolean queueRecovered, boolean holdLogInZK) {
159 String key = log.getName();
160 LOG.info("Going to report log #" + key + " for position " + position + " in " + log);
161 this.zkHelper.writeReplicationStatus(key, id, position);
162 if (holdLogInZK) {
163 return;
164 }
165 cleanOldLogs(key, id, queueRecovered);
166 }
167
168
169
170
171
172
173
174
175 public void cleanOldLogs(String key,
176 String id,
177 boolean queueRecovered) {
178 synchronized (this.hlogsById) {
179 SortedSet<String> hlogs = this.hlogsById.get(id);
180 if (queueRecovered || hlogs.first().equals(key)) {
181 return;
182 }
183 SortedSet<String> hlogSet = hlogs.headSet(key);
184 for (String hlog : hlogSet) {
185 this.zkHelper.removeLogFromList(hlog, id);
186 }
187 hlogSet.clear();
188 }
189 }
190
191
192
193
194
195 public void init() throws IOException {
196 for (String id : this.zkHelper.getPeerClusters().keySet()) {
197 addSource(id);
198 }
199 List<String> currentReplicators = this.zkHelper.getListOfReplicators();
200 if (currentReplicators == null || currentReplicators.size() == 0) {
201 return;
202 }
203 synchronized (otherRegionServers) {
204 refreshOtherRegionServersList();
205 LOG.info("Current list of replicators: " + currentReplicators
206 + " other RSs: " + otherRegionServers);
207 }
208
209 for (String rs : currentReplicators) {
210 synchronized (otherRegionServers) {
211 if (!this.otherRegionServers.contains(rs)) {
212 transferQueues(rs);
213 }
214 }
215 }
216 }
217
218
219
220
221
222
223
224 public ReplicationSourceInterface addSource(String id) throws IOException {
225 ReplicationSourceInterface src =
226 getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
227 synchronized (this.hlogsById) {
228 this.sources.add(src);
229 this.hlogsById.put(id, new TreeSet<String>());
230
231 if (this.latestPath != null) {
232 String name = this.latestPath.getName();
233 this.hlogsById.get(id).add(name);
234 try {
235 this.zkHelper.addLogToList(name, src.getPeerClusterZnode());
236 } catch (KeeperException ke) {
237 String message = "Cannot add log to zk for" +
238 " replication when creating a new source";
239 stopper.stop(message);
240 throw new IOException(message, ke);
241 }
242 src.enqueueLog(this.latestPath);
243 }
244 }
245 src.startup();
246 return src;
247 }
248
249
250
251
252 public void join() {
253 this.executor.shutdown();
254 if (this.sources.size() == 0) {
255 this.zkHelper.deleteOwnRSZNode();
256 }
257 for (ReplicationSourceInterface source : this.sources) {
258 source.terminate("Region server is closing");
259 }
260 }
261
262
263
264
265
266 protected Map<String, SortedSet<String>> getHLogs() {
267 return Collections.unmodifiableMap(hlogsById);
268 }
269
270
271
272
273
274 public List<ReplicationSourceInterface> getSources() {
275 return this.sources;
276 }
277
278
279
280
281
282 public List<ReplicationSourceInterface> getOldSources() {
283 return this.oldsources;
284 }
285
286 void preLogRoll(Path newLog) throws IOException {
287 if (!this.replicating.get()) {
288 LOG.warn("Replication stopped, won't add new log");
289 return;
290 }
291
292 synchronized (this.hlogsById) {
293 String name = newLog.getName();
294 for (ReplicationSourceInterface source : this.sources) {
295 try {
296 this.zkHelper.addLogToList(name, source.getPeerClusterZnode());
297 } catch (KeeperException ke) {
298 throw new IOException("Cannot add log to zk for replication", ke);
299 }
300 }
301 for (SortedSet<String> hlogs : this.hlogsById.values()) {
302 if (this.sources.isEmpty()) {
303
304
305 hlogs.clear();
306 }
307 hlogs.add(name);
308 }
309 }
310
311 this.latestPath = newLog;
312 }
313
314 void postLogRoll(Path newLog) throws IOException {
315 if (!this.replicating.get()) {
316 LOG.warn("Replication stopped, won't add new log");
317 return;
318 }
319
320
321 for (ReplicationSourceInterface source : this.sources) {
322 source.enqueueLog(newLog);
323 }
324 }
325
326
327
328
329
330 public ReplicationZookeeper getRepZkWrapper() {
331 return zkHelper;
332 }
333
334
335
336
337
338
339
340
341
342
343
344
345 public ReplicationSourceInterface getReplicationSource(
346 final Configuration conf,
347 final FileSystem fs,
348 final ReplicationSourceManager manager,
349 final Stoppable stopper,
350 final AtomicBoolean replicating,
351 final String peerId) throws IOException {
352 ReplicationSourceInterface src;
353 try {
354 @SuppressWarnings("rawtypes")
355 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
356 ReplicationSource.class.getCanonicalName()));
357 src = (ReplicationSourceInterface) c.newInstance();
358 } catch (Exception e) {
359 LOG.warn("Passed replication source implementation throws errors, " +
360 "defaulting to ReplicationSource", e);
361 src = new ReplicationSource();
362
363 }
364 src.init(conf, fs, manager, stopper, replicating, peerId);
365 return src;
366 }
367
368
369
370
371
372
373
374
375
376 public void transferQueues(String rsZnode) {
377 NodeFailoverWorker transfer = new NodeFailoverWorker(rsZnode);
378 try {
379 this.executor.execute(transfer);
380 } catch (RejectedExecutionException ex) {
381 LOG.info("Cancelling the transfer of " + rsZnode +
382 " because of " + ex.getMessage());
383 }
384 }
385
386
387
388
389
390 public void closeRecoveredQueue(ReplicationSourceInterface src) {
391 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
392 this.oldsources.remove(src);
393 this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
394 }
395
396
397
398
399
400
401 public void removePeer(String id) {
402 LOG.info("Closing the following queue " + id + ", currently have "
403 + sources.size() + " and another "
404 + oldsources.size() + " that were recovered");
405 String terminateMessage = "Replication stream was removed by a user";
406 ReplicationSourceInterface srcToRemove = null;
407 List<ReplicationSourceInterface> oldSourcesToDelete =
408 new ArrayList<ReplicationSourceInterface>();
409
410 for (ReplicationSourceInterface src : oldsources) {
411 if (id.equals(src.getPeerClusterId())) {
412 oldSourcesToDelete.add(src);
413 }
414 }
415 for (ReplicationSourceInterface src : oldSourcesToDelete) {
416 src.terminate(terminateMessage);
417 closeRecoveredQueue((src));
418 }
419 LOG.info("Number of deleted recovered sources for " + id + ": "
420 + oldSourcesToDelete.size());
421
422 for (ReplicationSourceInterface src : this.sources) {
423 if (id.equals(src.getPeerClusterId())) {
424 srcToRemove = src;
425 break;
426 }
427 }
428 if (srcToRemove == null) {
429 LOG.error("The queue we wanted to close is missing " + id);
430 return;
431 }
432 srcToRemove.terminate(terminateMessage);
433 this.sources.remove(srcToRemove);
434 this.zkHelper.deleteSource(id, true);
435 }
436
437
438
439
440
441
442
443
444
445 private boolean refreshOtherRegionServersList() {
446 List<String> newRsList = zkHelper.getRegisteredRegionServers();
447 if (newRsList == null) {
448 return false;
449 } else {
450 synchronized (otherRegionServers) {
451 otherRegionServers.clear();
452 otherRegionServers.addAll(newRsList);
453 }
454 }
455 return true;
456 }
457
458
459
460
461
462
463 public class OtherRegionServerWatcher extends ZooKeeperListener {
464
465
466
467
468 public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
469 super(watcher);
470 }
471
472
473
474
475
476 public void nodeCreated(String path) {
477 refreshListIfRightPath(path);
478 }
479
480
481
482
483
484 public void nodeDeleted(String path) {
485 if (stopper.isStopped()) {
486 return;
487 }
488 boolean cont = refreshListIfRightPath(path);
489 if (!cont) {
490 return;
491 }
492 LOG.info(path + " znode expired, trying to lock it");
493 transferQueues(ReplicationZookeeper.getZNodeName(path));
494 }
495
496
497
498
499
500 public void nodeChildrenChanged(String path) {
501 if (stopper.isStopped()) {
502 return;
503 }
504 refreshListIfRightPath(path);
505 }
506
507 private boolean refreshListIfRightPath(String path) {
508 if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
509 return false;
510 }
511 return refreshOtherRegionServersList();
512 }
513 }
514
515
516
517
518 public class PeersWatcher extends ZooKeeperListener {
519
520
521
522
523 public PeersWatcher(ZooKeeperWatcher watcher) {
524 super(watcher);
525 }
526
527
528
529
530
531 public void nodeDeleted(String path) {
532 List<String> peers = refreshPeersList(path);
533 if (peers == null) {
534 return;
535 }
536 String id = ReplicationZookeeper.getZNodeName(path);
537 removePeer(id);
538 }
539
540
541
542
543
544 public void nodeChildrenChanged(String path) {
545 List<String> peers = refreshPeersList(path);
546 if (peers == null) {
547 return;
548 }
549 for (String id : peers) {
550 try {
551 boolean added = zkHelper.connectToPeer(id);
552 if (added) {
553 addSource(id);
554 }
555 } catch (IOException e) {
556
557 LOG.error("Error while adding a new peer", e);
558 } catch (KeeperException e) {
559 LOG.error("Error while adding a new peer", e);
560 }
561 }
562 }
563
564
565
566
567
568
569
570
571 private List<String> refreshPeersList(String path) {
572 if (!path.startsWith(zkHelper.getPeersZNode())) {
573 return null;
574 }
575 return zkHelper.listPeersIdsAndWatch();
576 }
577 }
578
579
580
581
582
583 class NodeFailoverWorker extends Thread {
584
585 private String rsZnode;
586
587
588
589
590
591 public NodeFailoverWorker(String rsZnode) {
592 super("Failover-for-"+rsZnode);
593 this.rsZnode = rsZnode;
594 }
595
596 @Override
597 public void run() {
598
599 if (zkHelper.isThisOurZnode(this.rsZnode)) {
600 return;
601 }
602
603
604 try {
605 Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
606 } catch (InterruptedException e) {
607 LOG.warn("Interrupted while waiting before transferring a queue.");
608 Thread.currentThread().interrupt();
609 }
610
611 if (stopper.isStopped()) {
612 LOG.info("Not transferring queue since we are shutting down");
613 return;
614 }
615 SortedMap<String, SortedSet<String>> newQueues = null;
616
617
618 if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
619 LOG.info("Atomically moving " + rsZnode + "'s hlogs to my queue");
620 newQueues = zkHelper.copyQueuesFromRSUsingMulti(rsZnode);
621 } else {
622 LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
623 if (!zkHelper.lockOtherRS(rsZnode)) {
624 return;
625 }
626 newQueues = zkHelper.copyQueuesFromRS(rsZnode);
627 zkHelper.deleteRsQueues(rsZnode);
628 }
629
630 if (newQueues.isEmpty()) {
631 return;
632 }
633
634 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
635 String peerId = entry.getKey();
636 try {
637 ReplicationSourceInterface src = getReplicationSource(conf,
638 fs, ReplicationSourceManager.this, stopper, replicating, peerId);
639 if (!zkHelper.getPeerClusters().containsKey(src.getPeerClusterId())) {
640 src.terminate("Recovered queue doesn't belong to any current peer");
641 break;
642 }
643 oldsources.add(src);
644 for (String hlog : entry.getValue()) {
645 src.enqueueLog(new Path(oldLogDir, hlog));
646 }
647 src.startup();
648 } catch (IOException e) {
649
650 LOG.error("Failed creating a source", e);
651 }
652 }
653 }
654 }
655
656
657
658
659
660 public Path getOldLogDir() {
661 return this.oldLogDir;
662 }
663
664
665
666
667
668 public Path getLogDir() {
669 return this.logDir;
670 }
671
672
673
674
675
676 public FileSystem getFs() {
677 return this.fs;
678 }
679 }