1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication.regionserver;
20
21 import java.io.EOFException;
22 import java.io.FileNotFoundException;
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.Comparator;
26 import java.util.List;
27 import java.util.UUID;
28 import java.util.concurrent.PriorityBlockingQueue;
29 import java.util.concurrent.TimeUnit;
30
31 import org.apache.commons.lang.StringUtils;
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.hbase.classification.InterfaceAudience;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.fs.FileStatus;
37 import org.apache.hadoop.fs.FileSystem;
38 import org.apache.hadoop.fs.Path;
39 import org.apache.hadoop.hbase.HConstants;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.Stoppable;
42 import org.apache.hadoop.hbase.regionserver.wal.HLog;
43 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
44 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
45 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
46 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
47 import org.apache.hadoop.hbase.replication.ReplicationException;
48 import org.apache.hadoop.hbase.replication.ReplicationPeers;
49 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
50 import org.apache.hadoop.hbase.replication.ReplicationQueues;
51 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
52 import org.apache.hadoop.hbase.replication.WALEntryFilter;
53 import org.apache.hadoop.hbase.util.Threads;
54 import com.google.common.collect.Lists;
55 import com.google.common.util.concurrent.ListenableFuture;
56 import com.google.common.util.concurrent.Service;
57
58
59
60
61
62
63
64
65
66
67
68
69
70 @InterfaceAudience.Private
71 public class ReplicationSource extends Thread
72 implements ReplicationSourceInterface {
73
74 public static final Log LOG = LogFactory.getLog(ReplicationSource.class);
75
76 private PriorityBlockingQueue<Path> queue;
77 private ReplicationQueues replicationQueues;
78 private ReplicationPeers replicationPeers;
79
80 private Configuration conf;
81 private ReplicationQueueInfo replicationQueueInfo;
82
83 private String peerId;
84
85 private ReplicationSourceManager manager;
86
87 private Stoppable stopper;
88
89 private long sleepForRetries;
90
91 private long replicationQueueSizeCapacity;
92
93 private int replicationQueueNbCapacity;
94
95 private HLog.Reader reader;
96
97 private long lastLoggedPosition = -1;
98
99 private volatile Path currentPath;
100 private FileSystem fs;
101
102 private UUID clusterId;
103
104 private UUID peerClusterId;
105
106 private long totalReplicatedEdits = 0;
107
108 private long totalReplicatedOperations = 0;
109
110 private String peerClusterZnode;
111
112 private int maxRetriesMultiplier;
113
114 private int currentNbOperations = 0;
115
116 private int currentSize = 0;
117
118 private volatile boolean running = true;
119
120 private MetricsSource metrics;
121
122 private ReplicationHLogReaderManager repLogReader;
123
124 private int logQueueWarnThreshold;
125
126 private ReplicationEndpoint replicationEndpoint;
127
128 private WALEntryFilter walEntryFilter;
129
130 private ReplicationThrottler throttler;
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145 @Override
146 public void init(final Configuration conf, final FileSystem fs,
147 final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
148 final ReplicationPeers replicationPeers, final Stoppable stopper,
149 final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
150 final MetricsSource metrics)
151 throws IOException {
152 this.stopper = stopper;
153 this.conf = conf;
154 decorateConf();
155 this.replicationQueueSizeCapacity =
156 this.conf.getLong("replication.source.size.capacity", 1024*1024*64);
157 this.replicationQueueNbCapacity =
158 this.conf.getInt("replication.source.nb.capacity", 25000);
159 this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 10);
160 this.queue =
161 new PriorityBlockingQueue<Path>(
162 this.conf.getInt("hbase.regionserver.maxlogs", 32),
163 new LogsComparator());
164 long bandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
165 this.throttler = new ReplicationThrottler((double)bandwidth/10.0);
166 this.replicationQueues = replicationQueues;
167 this.replicationPeers = replicationPeers;
168 this.manager = manager;
169 this.sleepForRetries =
170 this.conf.getLong("replication.source.sleepforretries", 1000);
171 this.fs = fs;
172 this.metrics = metrics;
173 this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf);
174 this.clusterId = clusterId;
175
176 this.peerClusterZnode = peerClusterZnode;
177 this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
178
179 this.peerId = this.replicationQueueInfo.getPeerId();
180 this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
181 this.replicationEndpoint = replicationEndpoint;
182 }
183
184 private void decorateConf() {
185 String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
186 if (StringUtils.isNotEmpty(replicationCodec)) {
187 this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
188 }
189 }
190
191 @Override
192 public void enqueueLog(Path log) {
193 this.queue.put(log);
194 int queueSize = queue.size();
195 this.metrics.setSizeOfLogQueue(queueSize);
196
197 if (queueSize > this.logQueueWarnThreshold) {
198 LOG.warn("Queue size: " + queueSize +
199 " exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
200 }
201 }
202
203 private void uninitialize() {
204 LOG.debug("Source exiting " + this.peerId);
205 metrics.clear();
206 if (replicationEndpoint.state() == Service.State.STARTING
207 || replicationEndpoint.state() == Service.State.RUNNING) {
208 replicationEndpoint.stopAndWait();
209 }
210 }
211
212 @Override
213 public void run() {
214
215 if (!this.isActive()) {
216 uninitialize();
217 return;
218 }
219
220 try {
221
222 Service.State state = replicationEndpoint.start().get();
223 if (state != Service.State.RUNNING) {
224 LOG.warn("ReplicationEndpoint was not started. Exiting");
225 uninitialize();
226 return;
227 }
228 } catch (Exception ex) {
229 LOG.warn("Error starting ReplicationEndpoint, exiting", ex);
230 throw new RuntimeException(ex);
231 }
232
233
234 ArrayList<WALEntryFilter> filters = Lists.newArrayList(
235 (WALEntryFilter)new SystemTableWALEntryFilter());
236 WALEntryFilter filterFromEndpoint = this.replicationEndpoint.getWALEntryfilter();
237 if (filterFromEndpoint != null) {
238 filters.add(filterFromEndpoint);
239 }
240 this.walEntryFilter = new ChainWALEntryFilter(filters);
241
242 int sleepMultiplier = 1;
243
244 while (this.isActive() && this.peerClusterId == null) {
245 this.peerClusterId = replicationEndpoint.getPeerUUID();
246 if (this.isActive() && this.peerClusterId == null) {
247 if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
248 sleepMultiplier++;
249 }
250 }
251 }
252
253 if (!this.isActive()) {
254 uninitialize();
255 return;
256 }
257
258
259 sleepMultiplier = 1;
260
261
262
263 if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
264 this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
265 + peerClusterId + " which is not allowed by ReplicationEndpoint:"
266 + replicationEndpoint.getClass().getName(), null, false);
267 }
268 LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
269
270
271
272 if (this.replicationQueueInfo.isQueueRecovered()) {
273 try {
274 this.repLogReader.setPosition(this.replicationQueues.getLogPosition(this.peerClusterZnode,
275 this.queue.peek().getName()));
276 if (LOG.isTraceEnabled()) {
277 LOG.trace("Recovered queue started with log " + this.queue.peek() +
278 " at position " + this.repLogReader.getPosition());
279 }
280 } catch (ReplicationException e) {
281 this.terminate("Couldn't get the position of this recovered queue " +
282 this.peerClusterZnode, e);
283 }
284 }
285
286 while (isActive()) {
287
288 if (!isPeerEnabled()) {
289 if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
290 sleepMultiplier++;
291 }
292 continue;
293 }
294 Path oldPath = getCurrentPath();
295
296
297
298 boolean hasCurrentPath = getNextPath();
299 if (getCurrentPath() != null && oldPath == null) {
300 sleepMultiplier = 1;
301 }
302 if (!hasCurrentPath) {
303 if (sleepForRetries("No log to process", sleepMultiplier)) {
304 sleepMultiplier++;
305 }
306 continue;
307 }
308 boolean currentWALisBeingWrittenTo = false;
309
310
311
312
313
314
315
316
317 if (!this.replicationQueueInfo.isQueueRecovered() && queue.size() == 0) {
318 currentWALisBeingWrittenTo = true;
319 }
320
321 if (!openReader(sleepMultiplier)) {
322
323 sleepMultiplier = 1;
324 continue;
325 }
326
327
328 if (this.reader == null) {
329 if (sleepForRetries("Unable to open a reader", sleepMultiplier)) {
330 sleepMultiplier++;
331 }
332 continue;
333 }
334
335 boolean gotIOE = false;
336 currentNbOperations = 0;
337 List<HLog.Entry> entries = new ArrayList<HLog.Entry>(1);
338 currentSize = 0;
339 try {
340 if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo, entries)) {
341 continue;
342 }
343 } catch (IOException ioe) {
344 LOG.warn(this.peerClusterZnode + " Got: ", ioe);
345 gotIOE = true;
346 if (ioe.getCause() instanceof EOFException) {
347
348 boolean considerDumping = false;
349 if (this.replicationQueueInfo.isQueueRecovered()) {
350 try {
351 FileStatus stat = this.fs.getFileStatus(this.currentPath);
352 if (stat.getLen() == 0) {
353 LOG.warn(this.peerClusterZnode + " Got EOF and the file was empty");
354 }
355 considerDumping = true;
356 } catch (IOException e) {
357 LOG.warn(this.peerClusterZnode + " Got while getting file size: ", e);
358 }
359 }
360
361 if (considerDumping &&
362 sleepMultiplier == this.maxRetriesMultiplier &&
363 processEndOfFile()) {
364 continue;
365 }
366 }
367 } finally {
368 try {
369 this.reader = null;
370 this.repLogReader.closeReader();
371 } catch (IOException e) {
372 gotIOE = true;
373 LOG.warn("Unable to finalize the tailing of a file", e);
374 }
375 }
376
377
378
379
380 if (this.isActive() && (gotIOE || entries.isEmpty())) {
381 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
382 this.manager.logPositionAndCleanOldLogs(this.currentPath,
383 this.peerClusterZnode, this.repLogReader.getPosition(),
384 this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
385 this.lastLoggedPosition = this.repLogReader.getPosition();
386 }
387
388 if (!gotIOE) {
389 sleepMultiplier = 1;
390
391
392 this.metrics.setAgeOfLastShippedOp(System.currentTimeMillis());
393 }
394 if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
395 sleepMultiplier++;
396 }
397 continue;
398 }
399 sleepMultiplier = 1;
400 shipEdits(currentWALisBeingWrittenTo, entries);
401 }
402 uninitialize();
403 }
404
405
406
407
408
409
410
411
412
413
414 protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo,
415 List<HLog.Entry> entries) throws IOException{
416 long seenEntries = 0;
417 if (LOG.isTraceEnabled()) {
418 LOG.trace("Seeking in " + this.currentPath + " at position "
419 + this.repLogReader.getPosition());
420 }
421 this.repLogReader.seek();
422 long positionBeforeRead = this.repLogReader.getPosition();
423 HLog.Entry entry =
424 this.repLogReader.readNextAndSetPosition();
425 while (entry != null) {
426 this.metrics.incrLogEditsRead();
427 seenEntries++;
428
429
430 if (replicationEndpoint.canReplicateToSameCluster()
431 || !entry.getKey().getClusterIds().contains(peerClusterId)) {
432
433 entry = walEntryFilter.filter(entry);
434 WALEdit edit = null;
435 HLogKey logKey = null;
436 if (entry != null) {
437 edit = entry.getEdit();
438 logKey = entry.getKey();
439 }
440
441 if (edit != null && edit.size() != 0) {
442
443 logKey.addClusterId(clusterId);
444 currentNbOperations += countDistinctRowKeys(edit);
445 entries.add(entry);
446 currentSize += entry.getEdit().heapSize();
447 } else {
448 this.metrics.incrLogEditsFiltered();
449 }
450 }
451
452 if (currentSize >= this.replicationQueueSizeCapacity ||
453 entries.size() >= this.replicationQueueNbCapacity) {
454 break;
455 }
456 try {
457 entry = this.repLogReader.readNextAndSetPosition();
458 } catch (IOException ie) {
459 LOG.debug("Break on IOE: " + ie.getMessage());
460 break;
461 }
462 }
463 metrics.incrLogReadInBytes(this.repLogReader.getPosition() - positionBeforeRead);
464 if (currentWALisBeingWrittenTo) {
465 return false;
466 }
467
468
469 return seenEntries == 0 && processEndOfFile();
470 }
471
472
473
474
475
476 protected boolean getNextPath() {
477 try {
478 if (this.currentPath == null) {
479 this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
480 this.metrics.setSizeOfLogQueue(queue.size());
481 if (this.currentPath != null) {
482 this.manager.cleanOldLogs(this.currentPath.getName(),
483 this.peerId,
484 this.replicationQueueInfo.isQueueRecovered());
485 if (LOG.isTraceEnabled()) {
486 LOG.trace("New log: " + this.currentPath);
487 }
488 }
489 }
490 } catch (InterruptedException e) {
491 LOG.warn("Interrupted while reading edits", e);
492 }
493 return this.currentPath != null;
494 }
495
496
497
498
499
500
501
502 protected boolean openReader(int sleepMultiplier) {
503 try {
504 try {
505 if (LOG.isTraceEnabled()) {
506 LOG.trace("Opening log " + this.currentPath);
507 }
508 this.reader = repLogReader.openReader(this.currentPath);
509 } catch (FileNotFoundException fnfe) {
510 if (this.replicationQueueInfo.isQueueRecovered()) {
511
512
513
514 List<String> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
515 LOG.info("NB dead servers : " + deadRegionServers.size());
516 for (String curDeadServerName : deadRegionServers) {
517 Path deadRsDirectory =
518 new Path(manager.getLogDir().getParent(), curDeadServerName);
519 Path[] locs = new Path[] {
520 new Path(deadRsDirectory, currentPath.getName()),
521 new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT),
522 currentPath.getName()),
523 };
524 for (Path possibleLogLocation : locs) {
525 LOG.info("Possible location " + possibleLogLocation.toUri().toString());
526 if (this.manager.getFs().exists(possibleLogLocation)) {
527
528 LOG.info("Log " + this.currentPath + " still exists at " +
529 possibleLogLocation);
530
531 return true;
532 }
533 }
534 }
535
536
537 if (stopper instanceof ReplicationSyncUp.DummyServer) {
538 FileStatus[] rss = fs.listStatus(manager.getLogDir());
539 for (FileStatus rs : rss) {
540 Path p = rs.getPath();
541 FileStatus[] logs = fs.listStatus(p);
542 for (FileStatus log : logs) {
543 p = new Path(p, log.getPath().getName());
544 if (p.getName().equals(currentPath.getName())) {
545 currentPath = p;
546 LOG.info("Log " + this.currentPath + " exists under " + manager.getLogDir());
547
548 this.openReader(sleepMultiplier);
549 return true;
550 }
551 }
552 }
553 }
554
555
556
557
558
559
560
561
562 throw new IOException("File from recovered queue is " +
563 "nowhere to be found", fnfe);
564 } else {
565
566 Path archivedLogLocation =
567 new Path(manager.getOldLogDir(), currentPath.getName());
568 if (this.manager.getFs().exists(archivedLogLocation)) {
569 currentPath = archivedLogLocation;
570 LOG.info("Log " + this.currentPath + " was moved to " +
571 archivedLogLocation);
572
573 this.openReader(sleepMultiplier);
574
575 }
576
577 }
578 }
579 } catch (IOException ioe) {
580 if (ioe instanceof EOFException && isCurrentLogEmpty()) return true;
581 LOG.warn(this.peerClusterZnode + " Got: ", ioe);
582 this.reader = null;
583 if (ioe.getCause() instanceof NullPointerException) {
584
585
586
587 LOG.warn("Got NPE opening reader, will retry.");
588 } else if (sleepMultiplier == this.maxRetriesMultiplier) {
589
590
591 LOG.warn("Waited too long for this file, considering dumping");
592 return !processEndOfFile();
593 }
594 }
595 return true;
596 }
597
598
599
600
601
602
603
604 private boolean isCurrentLogEmpty() {
605 return (this.repLogReader.getPosition() == 0 &&
606 !this.replicationQueueInfo.isQueueRecovered() && queue.size() == 0);
607 }
608
609
610
611
612
613
614
615 protected boolean sleepForRetries(String msg, int sleepMultiplier) {
616 try {
617 if (LOG.isTraceEnabled()) {
618 LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
619 }
620 Thread.sleep(this.sleepForRetries * sleepMultiplier);
621 } catch (InterruptedException e) {
622 LOG.debug("Interrupted while sleeping between retries");
623 Thread.currentThread().interrupt();
624 }
625 return sleepMultiplier < maxRetriesMultiplier;
626 }
627
628
629
630
631
632
633
634 private int countDistinctRowKeys(WALEdit edit) {
635 List<KeyValue> kvs = edit.getKeyValues();
636 int distinctRowKeys = 1;
637 KeyValue lastKV = kvs.get(0);
638 for (int i = 0; i < edit.size(); i++) {
639 if (!kvs.get(i).matchingRow(lastKV)) {
640 distinctRowKeys++;
641 }
642 }
643 return distinctRowKeys;
644 }
645
646
647
648
649
650
651 protected void shipEdits(boolean currentWALisBeingWrittenTo, List<HLog.Entry> entries) {
652 int sleepMultiplier = 0;
653 if (entries.isEmpty()) {
654 LOG.warn("Was given 0 edits to ship");
655 return;
656 }
657 while (this.isActive()) {
658 try {
659 if (this.throttler.isEnabled()) {
660 long sleepTicks = this.throttler.getNextSleepInterval(currentSize);
661 if (sleepTicks > 0) {
662 try {
663 if (LOG.isTraceEnabled()) {
664 LOG.trace("To sleep " + sleepTicks + "ms for throttling control");
665 }
666 Thread.sleep(sleepTicks);
667 } catch (InterruptedException e) {
668 LOG.debug("Interrupted while sleeping for throttling control");
669 Thread.currentThread().interrupt();
670
671
672 continue;
673 }
674
675 this.throttler.resetStartTick();
676 }
677 }
678
679 ReplicationEndpoint.ReplicateContext replicateContext = new ReplicationEndpoint.ReplicateContext();
680 replicateContext.setEntries(entries).setSize(currentSize);
681
682
683 boolean replicated = replicationEndpoint.replicate(replicateContext);
684
685 if (!replicated) {
686 continue;
687 } else {
688 sleepMultiplier = Math.max(sleepMultiplier-1, 0);
689 }
690
691 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
692 this.manager.logPositionAndCleanOldLogs(this.currentPath,
693 this.peerClusterZnode, this.repLogReader.getPosition(),
694 this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
695 this.lastLoggedPosition = this.repLogReader.getPosition();
696 }
697 if (this.throttler.isEnabled()) {
698 this.throttler.addPushSize(currentSize);
699 }
700 this.totalReplicatedEdits += entries.size();
701 this.totalReplicatedOperations += currentNbOperations;
702 this.metrics.shipBatch(this.currentNbOperations, this.currentSize/1024);
703 this.metrics.setAgeOfLastShippedOp(entries.get(entries.size()-1).getKey().getWriteTime());
704 if (LOG.isTraceEnabled()) {
705 LOG.trace("Replicated " + this.totalReplicatedEdits + " entries in total, or "
706 + this.totalReplicatedOperations + " operations");
707 }
708 break;
709 } catch (Exception ex) {
710 LOG.warn(replicationEndpoint.getClass().getName() + " threw unknown exception:" + ex);
711 if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) {
712 sleepMultiplier++;
713 }
714 }
715 }
716 }
717
718
719
720
721
722
723 protected boolean isPeerEnabled() {
724 return this.replicationPeers.getStatusOfPeer(this.peerId);
725 }
726
727
728
729
730
731
732
733
734 protected boolean processEndOfFile() {
735 if (this.queue.size() != 0) {
736 if (LOG.isTraceEnabled()) {
737 String filesize = "N/A";
738 try {
739 FileStatus stat = this.fs.getFileStatus(this.currentPath);
740 filesize = stat.getLen()+"";
741 } catch (IOException ex) {}
742 LOG.trace("Reached the end of a log, stats: " + getStats() +
743 ", and the length of the file is " + filesize);
744 }
745 this.currentPath = null;
746 this.repLogReader.finishCurrentFile();
747 this.reader = null;
748 return true;
749 } else if (this.replicationQueueInfo.isQueueRecovered()) {
750 this.manager.closeRecoveredQueue(this);
751 LOG.info("Finished recovering the queue with the following stats " + getStats());
752 this.running = false;
753 return true;
754 }
755 return false;
756 }
757
758 @Override
759 public void startup() {
760 String n = Thread.currentThread().getName();
761 Thread.UncaughtExceptionHandler handler =
762 new Thread.UncaughtExceptionHandler() {
763 @Override
764 public void uncaughtException(final Thread t, final Throwable e) {
765 LOG.error("Unexpected exception in ReplicationSource," +
766 " currentPath=" + currentPath, e);
767 }
768 };
769 Threads.setDaemonThreadRunning(
770 this, n + ".replicationSource," +
771 this.peerClusterZnode, handler);
772 }
773
774 @Override
775 public void terminate(String reason) {
776 terminate(reason, null);
777 }
778
779 @Override
780 public void terminate(String reason, Exception cause) {
781 terminate(reason, cause, true);
782 }
783
784 public void terminate(String reason, Exception cause, boolean join) {
785 if (cause == null) {
786 LOG.info("Closing source "
787 + this.peerClusterZnode + " because: " + reason);
788
789 } else {
790 LOG.error("Closing source " + this.peerClusterZnode
791 + " because an error occurred: " + reason, cause);
792 }
793 this.running = false;
794 this.interrupt();
795 ListenableFuture<Service.State> future = null;
796 if (this.replicationEndpoint != null) {
797 future = this.replicationEndpoint.stop();
798 }
799 if (join) {
800 Threads.shutdown(this, this.sleepForRetries);
801 if (future != null) {
802 try {
803 future.get();
804 } catch (Exception e) {
805 LOG.warn("Got exception:" + e);
806 }
807 }
808 }
809 }
810
811 @Override
812 public String getPeerClusterZnode() {
813 return this.peerClusterZnode;
814 }
815
816 @Override
817 public String getPeerClusterId() {
818 return this.peerId;
819 }
820
821 @Override
822 public Path getCurrentPath() {
823 return this.currentPath;
824 }
825
826 private boolean isActive() {
827 return !this.stopper.isStopped() && this.running && !isInterrupted();
828 }
829
830
831
832
833 public static class LogsComparator implements Comparator<Path> {
834
835 @Override
836 public int compare(Path o1, Path o2) {
837 return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
838 }
839
840
841
842
843
844
845
846 private long getTS(Path p) {
847 String[] parts = p.getName().split("\\.");
848 return Long.parseLong(parts[parts.length-1]);
849 }
850 }
851
852 @Override
853 public String getStats() {
854 long position = this.repLogReader.getPosition();
855 return "Total replicated edits: " + totalReplicatedEdits +
856 ", currently replicating from: " + this.currentPath +
857 " at position: " + position;
858 }
859
860
861
862
863
864 public MetricsSource getSourceMetrics() {
865 return this.metrics;
866 }
867 }