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