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