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