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