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.regionserver;
21
22 import com.google.common.collect.Sets;
23 import com.google.common.util.concurrent.ThreadFactoryBuilder;
24
25 import java.io.IOException;
26 import java.util.ArrayList;
27 import java.util.Collections;
28 import java.util.HashMap;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.Random;
32 import java.util.Set;
33 import java.util.SortedSet;
34 import java.util.TreeSet;
35 import java.util.UUID;
36 import java.util.concurrent.ConcurrentHashMap;
37 import java.util.concurrent.CopyOnWriteArrayList;
38 import java.util.concurrent.LinkedBlockingQueue;
39 import java.util.concurrent.RejectedExecutionException;
40 import java.util.concurrent.ThreadPoolExecutor;
41 import java.util.concurrent.TimeUnit;
42 import java.util.concurrent.atomic.AtomicLong;
43
44 import org.apache.commons.logging.Log;
45 import org.apache.commons.logging.LogFactory;
46 import org.apache.hadoop.conf.Configuration;
47 import org.apache.hadoop.fs.FileSystem;
48 import org.apache.hadoop.fs.Path;
49 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
50 import org.apache.hadoop.hbase.HConstants;
51 import org.apache.hadoop.hbase.Server;
52 import org.apache.hadoop.hbase.TableDescriptors;
53 import org.apache.hadoop.hbase.TableName;
54 import org.apache.hadoop.hbase.classification.InterfaceAudience;
55 import org.apache.hadoop.hbase.regionserver.HRegionServer;
56 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
57 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
58 import org.apache.hadoop.hbase.replication.ReplicationException;
59 import org.apache.hadoop.hbase.replication.ReplicationListener;
60 import org.apache.hadoop.hbase.replication.ReplicationPeer;
61 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
62 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
63 import org.apache.hadoop.hbase.replication.ReplicationPeers;
64 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
65 import org.apache.hadoop.hbase.replication.ReplicationQueues;
66 import org.apache.hadoop.hbase.replication.ReplicationSourceWithoutPeerException;
67 import org.apache.hadoop.hbase.replication.ReplicationTracker;
68 import org.apache.hadoop.hbase.util.Pair;
69 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87 @InterfaceAudience.Private
88 public class ReplicationSourceManager implements ReplicationListener {
89 private static final Log LOG =
90 LogFactory.getLog(ReplicationSourceManager.class);
91
92 private final List<ReplicationSourceInterface> sources;
93
94 private final List<ReplicationSourceInterface> oldsources;
95 private final ReplicationQueues replicationQueues;
96 private final ReplicationTracker replicationTracker;
97 private final ReplicationPeers replicationPeers;
98
99 private final UUID clusterId;
100
101 private final Server server;
102
103
104 private final Map<String, Map<String, SortedSet<String>>> walsById;
105
106 private final Map<String, Map<String, SortedSet<String>>> walsByIdRecoveredQueues;
107 private final Configuration conf;
108 private final FileSystem fs;
109
110 private final Map<String, Path> latestPaths;
111
112 private final Path logDir;
113
114 private final Path oldLogDir;
115
116 private final long sleepBeforeFailover;
117
118 private final ThreadPoolExecutor executor;
119
120 private final Random rand;
121 private final boolean replicationForBulkLoadDataEnabled;
122
123 private AtomicLong totalBufferUsed = new AtomicLong();
124
125
126
127
128
129
130
131
132
133
134
135
136
137 public ReplicationSourceManager(final ReplicationQueues replicationQueues,
138 final ReplicationPeers replicationPeers, final ReplicationTracker replicationTracker,
139 final Configuration conf, final Server server, final FileSystem fs, final Path logDir,
140 final Path oldLogDir, final UUID clusterId) {
141
142
143 this.sources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
144 this.replicationQueues = replicationQueues;
145 this.replicationPeers = replicationPeers;
146 this.replicationTracker = replicationTracker;
147 this.server = server;
148 this.walsById = new HashMap<String, Map<String, SortedSet<String>>>();
149 this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, Map<String, SortedSet<String>>>();
150 this.oldsources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
151 this.conf = conf;
152 this.fs = fs;
153 this.logDir = logDir;
154 this.oldLogDir = oldLogDir;
155 this.sleepBeforeFailover =
156 conf.getLong("replication.sleep.before.failover", 30000);
157 this.clusterId = clusterId;
158 this.replicationTracker.registerListener(this);
159 this.replicationPeers.getAllPeerIds();
160
161
162 int nbWorkers = conf.getInt("replication.executor.workers", 1);
163
164
165 this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers,
166 100, TimeUnit.MILLISECONDS,
167 new LinkedBlockingQueue<Runnable>());
168 ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
169 tfb.setNameFormat("ReplicationExecutor-%d");
170 tfb.setDaemon(true);
171 this.executor.setThreadFactory(tfb.build());
172 this.rand = new Random();
173 this.latestPaths = new HashMap<>();
174 replicationForBulkLoadDataEnabled =
175 conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
176 HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
177 }
178
179
180
181
182
183
184
185
186
187
188
189
190 public synchronized void logPositionAndCleanOldLogs(Path log, String id, long position,
191 boolean queueRecovered, boolean holdLogInZK) {
192 String fileName = log.getName();
193 this.replicationQueues.setLogPosition(id, fileName, position);
194 if (holdLogInZK) {
195 return;
196 }
197
198 cleanOldLogs(fileName, id, queueRecovered);
199 }
200
201
202
203
204
205
206
207
208 public void cleanOldLogs(String key, String id, boolean queueRecovered) {
209 String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(key);
210 if (queueRecovered) {
211 Map<String, SortedSet<String>> walsForPeer = walsByIdRecoveredQueues.get(id);
212 if(walsForPeer != null) {
213 SortedSet<String> wals = walsForPeer.get(logPrefix);
214 if (wals != null && !wals.first().equals(key)) {
215 cleanOldLogs(wals, key, id);
216 }
217 }
218 } else {
219 synchronized (this.walsById) {
220 SortedSet<String> wals = getLogsWithPrefix(id, logPrefix);
221 if (wals != null && !wals.first().equals(key)) {
222 cleanOldLogs(wals, key, id);
223 }
224 }
225 }
226 }
227
228 private void cleanOldLogs(SortedSet<String> wals, String key, String id) {
229 SortedSet<String> walSet = wals.headSet(key);
230 LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
231 try {
232 for (String wal : walSet) {
233 this.replicationQueues.removeLog(id, wal);
234 }
235 } catch (ReplicationSourceWithoutPeerException rspe) {
236
237
238 LOG.warn("Replication peer " + id + " has been removed and source is still running", rspe);
239 String peerId = id;
240 if (peerId.contains("-")) {
241 peerId = peerId.split("-")[0];
242 }
243 schedulePeerRemoval(peerId);
244 }
245 walSet.clear();
246 }
247
248
249
250
251
252
253
254 public SortedSet<String> getLogsWithPrefix(String walGroupId, String logPrefix) {
255 return walsById.get(walGroupId).get(logPrefix);
256 }
257
258
259
260
261
262 protected void init() throws IOException, ReplicationException {
263 for (String id : this.replicationPeers.getPeerIds()) {
264 addSource(id);
265 if (replicationForBulkLoadDataEnabled) {
266
267
268 this.replicationQueues.addPeerToHFileRefs(id);
269 }
270 }
271 List<String> currentReplicators = this.replicationQueues.getListOfReplicators();
272 if (currentReplicators == null || currentReplicators.isEmpty()) {
273 return;
274 }
275 List<String> otherRegionServers = replicationTracker.getListOfRegionServers();
276 LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
277 + otherRegionServers);
278
279
280 for (String rs : currentReplicators) {
281 if (!otherRegionServers.contains(rs)) {
282 transferQueues(rs);
283 }
284 }
285 }
286
287
288
289
290
291
292
293
294 protected ReplicationSourceInterface addSource(String id) throws IOException,
295 ReplicationException {
296 ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
297 ReplicationPeer peer = replicationPeers.getPeer(id);
298 ReplicationSourceInterface src =
299 getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
300 this.replicationPeers, server, id, this.clusterId, peerConfig, peer);
301 synchronized (this.walsById) {
302 this.sources.add(src);
303 Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
304 this.walsById.put(id, walsByGroup);
305
306 synchronized (latestPaths) {
307 if (!latestPaths.isEmpty()) {
308 for (Map.Entry<String, Path> walPrefixAndPath : latestPaths.entrySet()) {
309 Path walPath = walPrefixAndPath.getValue();
310 SortedSet<String> wals = new TreeSet<>();
311 wals.add(walPath.getName());
312 walsByGroup.put(walPrefixAndPath.getKey(), wals);
313 try {
314 this.replicationQueues.addLog(id, walPath.getName());
315 } catch (ReplicationException e) {
316 String message =
317 "Cannot add log to queue when creating a new source, queueId=" + id
318 + ", filename=" + walPath.getName();
319 server.stop(message);
320 throw e;
321 }
322 src.enqueueLog(walPath);
323 }
324 }
325 }
326 }
327 src.startup();
328 return src;
329 }
330
331
332
333
334
335 public void deleteSource(String peerId, boolean closeConnection) {
336 this.replicationQueues.removeQueue(peerId);
337 if (closeConnection) {
338 this.replicationPeers.peerRemoved(peerId);
339 }
340 }
341
342
343
344
345 public void join() {
346 this.executor.shutdown();
347 if (this.sources.size() == 0) {
348 this.replicationQueues.removeAllQueues();
349 }
350 for (ReplicationSourceInterface source : this.sources) {
351 source.terminate("Region server is closing");
352 }
353 }
354
355
356
357
358
359 protected Map<String, Map<String, SortedSet<String>>> getWALs() {
360 return Collections.unmodifiableMap(walsById);
361 }
362
363
364
365
366
367 protected Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
368 return Collections.unmodifiableMap(walsByIdRecoveredQueues);
369 }
370
371
372
373
374
375 public List<ReplicationSourceInterface> getSources() {
376 return this.sources;
377 }
378
379
380
381
382
383 public List<ReplicationSourceInterface> getOldSources() {
384 return this.oldsources;
385 }
386
387
388
389
390
391
392 public ReplicationSourceInterface getSource(String peerId) {
393 for (ReplicationSourceInterface source: getSources()) {
394 if (source.getPeerClusterId().equals(peerId)) {
395 return source;
396 }
397 }
398 return null;
399 }
400
401 List<String> getAllQueues() {
402 return replicationQueues.getAllQueues();
403 }
404
405 void preLogRoll(Path newLog) throws IOException {
406 recordLog(newLog);
407 String logName = newLog.getName();
408 String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
409 synchronized (latestPaths) {
410 latestPaths.put(logPrefix, newLog);
411 }
412 }
413
414
415
416
417
418
419
420 private void recordLog(Path logPath) throws IOException {
421 String logName = logPath.getName();
422 String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
423
424
425 synchronized (replicationPeers) {
426 for (String id : replicationPeers.getPeerIds()) {
427 try {
428 this.replicationQueues.addLog(id, logName);
429 } catch (ReplicationException e) {
430 throw new IOException("Cannot add log to replication queue"
431 + " when creating a new source, queueId=" + id + ", filename=" + logName, e);
432 }
433 }
434 }
435
436 synchronized (walsById) {
437 for (Map.Entry<String, Map<String, SortedSet<String>>> entry : this.walsById.entrySet()) {
438 String peerId = entry.getKey();
439 Map<String, SortedSet<String>> walsByPrefix = entry.getValue();
440 boolean existingPrefix = false;
441 for (Map.Entry<String, SortedSet<String>> walsEntry : walsByPrefix.entrySet()) {
442 SortedSet<String> wals = walsEntry.getValue();
443 if (this.sources.isEmpty()) {
444
445
446 wals.clear();
447 }
448 if (logPrefix.equals(walsEntry.getKey())) {
449 wals.add(logName);
450 existingPrefix = true;
451 }
452 }
453 if (!existingPrefix) {
454
455 LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId);
456 SortedSet<String> wals = new TreeSet<String>();
457 wals.add(logName);
458 walsByPrefix.put(logPrefix, wals);
459 }
460 }
461 }
462 }
463
464 void postLogRoll(Path newLog) throws IOException {
465
466 for (ReplicationSourceInterface source : this.sources) {
467 source.enqueueLog(newLog);
468 }
469 }
470
471 public AtomicLong getTotalBufferUsed() {
472 return totalBufferUsed;
473 }
474
475
476
477
478
479
480
481
482
483
484
485 protected ReplicationSourceInterface getReplicationSource(final Configuration conf,
486 final FileSystem fs, final ReplicationSourceManager manager,
487 final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
488 final Server server, final String peerId, final UUID clusterId,
489 final ReplicationPeerConfig peerConfig, final ReplicationPeer replicationPeer)
490 throws IOException {
491 RegionServerCoprocessorHost rsServerHost = null;
492 TableDescriptors tableDescriptors = null;
493 if (server instanceof HRegionServer) {
494 rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost();
495 tableDescriptors = ((HRegionServer) server).getTableDescriptors();
496 }
497 ReplicationSourceInterface src;
498 try {
499 @SuppressWarnings("rawtypes")
500 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
501 ReplicationSource.class.getCanonicalName()));
502 src = (ReplicationSourceInterface) c.newInstance();
503 } catch (Exception e) {
504 LOG.warn("Passed replication source implementation throws errors, " +
505 "defaulting to ReplicationSource", e);
506 src = new ReplicationSource();
507 }
508
509 ReplicationEndpoint replicationEndpoint = null;
510 try {
511 String replicationEndpointImpl = peerConfig.getReplicationEndpointImpl();
512 if (replicationEndpointImpl == null) {
513
514 replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
515 }
516 @SuppressWarnings("rawtypes")
517 Class c = Class.forName(replicationEndpointImpl);
518 replicationEndpoint = (ReplicationEndpoint) c.newInstance();
519 if(rsServerHost != null) {
520 ReplicationEndpoint newReplicationEndPoint = rsServerHost
521 .postCreateReplicationEndPoint(replicationEndpoint);
522 if(newReplicationEndPoint != null) {
523
524 replicationEndpoint = newReplicationEndPoint;
525 }
526 }
527 } catch (Exception e) {
528 LOG.warn("Passed replication endpoint implementation throws errors"
529 + " while initializing ReplicationSource for peer: " + peerId, e);
530 throw new IOException(e);
531 }
532
533 MetricsSource metrics = new MetricsSource(peerId);
534
535 src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId,
536 clusterId, replicationEndpoint, metrics);
537
538
539 replicationEndpoint.init(new ReplicationEndpoint.Context(
540 conf, replicationPeer.getConfiguration(),
541 fs, peerId, clusterId, replicationPeer, metrics, tableDescriptors, server));
542
543 return src;
544 }
545
546
547
548
549
550
551
552
553
554 private void transferQueues(String rsZnode) {
555 NodeFailoverWorker transfer =
556 new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers,
557 this.clusterId);
558 try {
559 this.executor.execute(transfer);
560 } catch (RejectedExecutionException ex) {
561 CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
562 .getGlobalSource().incrFailedRecoveryQueue();
563 LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
564 }
565 }
566
567
568
569
570
571 public void closeRecoveredQueue(ReplicationSourceInterface src) {
572 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
573 src.getSourceMetrics().clear();
574 this.oldsources.remove(src);
575 deleteSource(src.getPeerClusterZnode(), false);
576 this.walsByIdRecoveredQueues.remove(src.getPeerClusterZnode());
577 }
578
579
580
581
582
583 public void closeQueue(ReplicationSourceInterface src) {
584 LOG.info("Done with the queue " + src.getPeerClusterZnode());
585 src.getSourceMetrics().clear();
586 this.sources.remove(src);
587 deleteSource(src.getPeerClusterZnode(), true);
588 this.walsById.remove(src.getPeerClusterZnode());
589 }
590
591
592
593
594
595
596 public void removePeer(String id) {
597 LOG.info("Closing the following queue " + id + ", currently have "
598 + sources.size() + " and another "
599 + oldsources.size() + " that were recovered");
600 String terminateMessage = "Replication stream was removed by a user";
601 List<ReplicationSourceInterface> oldSourcesToDelete =
602 new ArrayList<ReplicationSourceInterface>();
603
604
605 synchronized (oldsources) {
606
607 for (ReplicationSourceInterface src : oldsources) {
608 if (id.equals(src.getPeerClusterId())) {
609 oldSourcesToDelete.add(src);
610 }
611 }
612 for (ReplicationSourceInterface src : oldSourcesToDelete) {
613 src.terminate(terminateMessage);
614 closeRecoveredQueue(src);
615 }
616 }
617 LOG.info("Number of deleted recovered sources for " + id + ": "
618 + oldSourcesToDelete.size());
619
620 List<ReplicationSourceInterface> srcToRemove = new ArrayList<ReplicationSourceInterface>();
621
622 synchronized (this.replicationPeers) {
623 for (ReplicationSourceInterface src : this.sources) {
624 if (id.equals(src.getPeerClusterId())) {
625 srcToRemove.add(src);
626 }
627 }
628 if (srcToRemove.isEmpty()) {
629 LOG.error("The peer we wanted to remove is missing a ReplicationSourceInterface. " +
630 "This could mean that ReplicationSourceInterface initialization failed for this peer " +
631 "and that replication on this peer may not be caught up. peerId=" + id);
632 }
633 for (ReplicationSourceInterface toRemove : srcToRemove) {
634 toRemove.terminate(terminateMessage);
635 closeQueue(toRemove);
636 }
637 deleteSource(id, true);
638 }
639 }
640
641 @Override
642 public void regionServerRemoved(String regionserver) {
643 transferQueues(regionserver);
644 }
645
646
647
648
649
650
651
652
653
654 private void schedulePeerRemoval(final String peerId) {
655 LOG.info(String.format("Scheduling an async peer removal for peer %s", peerId));
656 this.executor.submit(new Runnable() {
657 @Override public void run() {
658 peerRemoved(peerId);
659 }
660 });
661 }
662
663 @Override
664 public void peerRemoved(String peerId) {
665 removePeer(peerId);
666 this.replicationQueues.removePeerFromHFileRefs(peerId);
667 }
668
669 @Override
670 public void peerListChanged(List<String> peerIds) {
671 for (String id : peerIds) {
672 try {
673 boolean added = this.replicationPeers.peerAdded(id);
674 if (added) {
675 addSource(id);
676 if (replicationForBulkLoadDataEnabled) {
677 this.replicationQueues.addPeerToHFileRefs(id);
678 }
679 }
680 } catch (Exception e) {
681 LOG.error("Error while adding a new peer", e);
682 }
683 }
684 }
685
686 Set<Path> getLastestPath() {
687 synchronized (latestPaths) {
688 return Sets.newHashSet(latestPaths.values());
689 }
690 }
691
692
693
694
695
696 class NodeFailoverWorker extends Thread {
697
698 private String rsZnode;
699 private final ReplicationQueues rq;
700 private final ReplicationPeers rp;
701 private final UUID clusterId;
702
703
704
705
706 public NodeFailoverWorker(String rsZnode) {
707 this(rsZnode, replicationQueues, replicationPeers, ReplicationSourceManager.this.clusterId);
708 }
709
710 public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues,
711 final ReplicationPeers replicationPeers, final UUID clusterId) {
712 super("Failover-for-"+rsZnode);
713 this.rsZnode = rsZnode;
714 this.rq = replicationQueues;
715 this.rp = replicationPeers;
716 this.clusterId = clusterId;
717 }
718
719 @Override
720 public void run() {
721 if (this.rq.isThisOurZnode(rsZnode)) {
722 return;
723 }
724
725
726 try {
727 Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
728 } catch (InterruptedException e) {
729 LOG.warn("Interrupted while waiting before transferring a queue.");
730 Thread.currentThread().interrupt();
731 }
732
733 if (server.isStopped()) {
734 LOG.info("Not transferring queue since we are shutting down");
735 return;
736 }
737
738 Map<String, SortedSet<String>> newQueues = new HashMap<>();
739 List<String> peers = rq.getUnClaimedQueueIds(rsZnode);
740 while (peers != null && !peers.isEmpty()) {
741 Pair<String, SortedSet<String>> peer = this.rq.claimQueue(rsZnode,
742 peers.get(rand.nextInt(peers.size())));
743 long sleep = sleepBeforeFailover/2;
744 if (peer != null) {
745 newQueues.put(peer.getFirst(), peer.getSecond());
746 sleep = sleepBeforeFailover;
747 }
748 try {
749 Thread.sleep(sleep);
750 } catch (InterruptedException e) {
751 LOG.warn("Interrupted while waiting before transferring a queue.");
752 Thread.currentThread().interrupt();
753 }
754 peers = rq.getUnClaimedQueueIds(rsZnode);
755 }
756 if (peers != null) {
757 rq.removeReplicatorIfQueueIsEmpty(rsZnode);
758 }
759
760
761 if (newQueues.isEmpty()) {
762
763
764 return;
765 }
766
767 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
768 String peerId = entry.getKey();
769 SortedSet<String> walsSet = entry.getValue();
770 try {
771
772 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
773 String actualPeerId = replicationQueueInfo.getPeerId();
774 ReplicationPeer peer = replicationPeers.getPeer(actualPeerId);
775 ReplicationPeerConfig peerConfig = null;
776 try {
777 peerConfig = replicationPeers.getReplicationPeerConfig(actualPeerId);
778 } catch (ReplicationException ex) {
779 LOG.warn("Received exception while getting replication peer config, skipping replay"
780 + ex);
781 }
782 if (peer == null || peerConfig == null) {
783 LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode);
784 replicationQueues.removeQueue(peerId);
785 continue;
786 }
787 if (server instanceof ReplicationSyncUp.DummyServer
788 && peer.getPeerState().equals(PeerState.DISABLED)) {
789 LOG.warn("Peer " + actualPeerId + " is disbaled. ReplicationSyncUp tool will skip "
790 + "replicating data to this peer.");
791 continue;
792 }
793
794 Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
795 walsByIdRecoveredQueues.put(peerId, walsByGroup);
796 for (String wal : walsSet) {
797 String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(wal);
798 SortedSet<String> wals = walsByGroup.get(walPrefix);
799 if (wals == null) {
800 wals = new TreeSet<String>();
801 walsByGroup.put(walPrefix, wals);
802 }
803 wals.add(wal);
804 }
805
806
807 ReplicationSourceInterface src =
808 getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
809 server, peerId, this.clusterId, peerConfig, peer);
810
811
812 synchronized (oldsources) {
813 if (!this.rp.getPeerIds().contains(src.getPeerClusterId())) {
814 src.terminate("Recovered queue doesn't belong to any current peer");
815 closeRecoveredQueue(src);
816 continue;
817 }
818 oldsources.add(src);
819 for (String wal : walsSet) {
820 src.enqueueLog(new Path(oldLogDir, wal));
821 }
822 src.startup();
823 }
824 } catch (IOException e) {
825
826 LOG.error("Failed creating a source", e);
827 }
828 }
829 }
830 }
831
832
833
834
835
836 public Path getOldLogDir() {
837 return this.oldLogDir;
838 }
839
840
841
842
843
844 public Path getLogDir() {
845 return this.logDir;
846 }
847
848
849
850
851
852 public FileSystem getFs() {
853 return this.fs;
854 }
855
856
857
858
859
860 public ReplicationPeers getReplicationPeers() {
861 return this.replicationPeers;
862 }
863
864
865
866
867 public String getStats() {
868 StringBuffer stats = new StringBuffer();
869 for (ReplicationSourceInterface source : sources) {
870 stats.append("Normal source for cluster " + source.getPeerClusterId() + ": ");
871 stats.append(source.getStats() + "\n");
872 }
873 for (ReplicationSourceInterface oldSource : oldsources) {
874 stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerClusterId()+": ");
875 stats.append(oldSource.getStats()+ "\n");
876 }
877 return stats.toString();
878 }
879
880 public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
881 throws ReplicationException {
882 for (ReplicationSourceInterface source : this.sources) {
883 source.addHFileRefs(tableName, family, pairs);
884 }
885 }
886
887 public void cleanUpHFileRefs(String peerId, List<String> files) {
888 this.replicationQueues.removeHFileRefs(peerId, files);
889 }
890 }