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