1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.replication.regionserver;
21
22 import java.io.EOFException;
23 import java.io.FileNotFoundException;
24 import java.io.IOException;
25 import java.net.ConnectException;
26 import java.net.SocketTimeoutException;
27 import java.util.ArrayList;
28 import java.util.Arrays;
29 import java.util.Collections;
30 import java.util.Comparator;
31 import java.util.HashSet;
32 import java.util.List;
33 import java.util.NavigableMap;
34 import java.util.Random;
35 import java.util.Set;
36 import java.util.UUID;
37 import java.util.concurrent.CountDownLatch;
38 import java.util.concurrent.PriorityBlockingQueue;
39 import java.util.concurrent.TimeUnit;
40 import java.util.concurrent.atomic.AtomicBoolean;
41
42 import org.apache.commons.logging.Log;
43 import org.apache.commons.logging.LogFactory;
44 import org.apache.hadoop.conf.Configuration;
45 import org.apache.hadoop.fs.FileStatus;
46 import org.apache.hadoop.fs.FileSystem;
47 import org.apache.hadoop.fs.Path;
48 import org.apache.hadoop.hbase.HConstants;
49 import org.apache.hadoop.hbase.KeyValue;
50 import org.apache.hadoop.hbase.ServerName;
51 import org.apache.hadoop.hbase.Stoppable;
52 import org.apache.hadoop.hbase.client.HConnection;
53 import org.apache.hadoop.hbase.client.HConnectionManager;
54 import org.apache.hadoop.hbase.ipc.HRegionInterface;
55 import org.apache.hadoop.hbase.regionserver.wal.HLog;
56 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
57 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
58 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
59 import org.apache.hadoop.hbase.util.Bytes;
60 import org.apache.hadoop.hbase.util.Threads;
61 import org.apache.hadoop.hbase.zookeeper.ClusterId;
62 import org.apache.hadoop.ipc.RemoteException;
63 import org.apache.zookeeper.KeeperException;
64
65
66
67
68
69
70
71
72
73
74
75
76
77 public class ReplicationSource extends Thread
78 implements ReplicationSourceInterface {
79
80 private static final Log LOG = LogFactory.getLog(ReplicationSource.class);
81
82 private PriorityBlockingQueue<Path> queue;
83
84 private HLog.Entry[] entriesArray;
85 private HConnection conn;
86
87 private ReplicationZookeeper zkHelper;
88 private Configuration conf;
89
90 private float ratio;
91 private Random random;
92
93 private AtomicBoolean replicating;
94
95 private String peerId;
96
97 private ReplicationSourceManager manager;
98
99 private Stoppable stopper;
100
101 private List<ServerName> currentPeers;
102
103 private long sleepForRetries;
104
105 private long replicationQueueSizeCapacity;
106
107 private int replicationQueueNbCapacity;
108
109 private HLog.Reader reader;
110
111 private long lastLoggedPosition = -1;
112
113 private volatile Path currentPath;
114 private FileSystem fs;
115
116 private UUID clusterId;
117
118 private UUID peerClusterId;
119
120 private long totalReplicatedEdits = 0;
121
122 private String peerClusterZnode;
123
124 private boolean queueRecovered;
125
126 private List<String> deadRegionServers = new ArrayList<String>();
127
128 private int maxRetriesMultiplier;
129
130 private int socketTimeoutMultiplier;
131
132 private int currentNbEntries = 0;
133
134 private int currentNbOperations = 0;
135
136 private int currentSize = 0;
137
138 private volatile boolean running = true;
139
140 private ReplicationSourceMetrics metrics;
141
142 private ReplicationHLogReaderManager repLogReader;
143
144
145
146
147
148
149
150
151
152
153
154
155
156 public void init(final Configuration conf,
157 final FileSystem fs,
158 final ReplicationSourceManager manager,
159 final Stoppable stopper,
160 final AtomicBoolean replicating,
161 final String peerClusterZnode)
162 throws IOException {
163 this.stopper = stopper;
164 this.conf = conf;
165 this.replicationQueueSizeCapacity =
166 this.conf.getLong("replication.source.size.capacity", 1024*1024*64);
167 this.replicationQueueNbCapacity =
168 this.conf.getInt("replication.source.nb.capacity", 25000);
169 this.entriesArray = new HLog.Entry[this.replicationQueueNbCapacity];
170 for (int i = 0; i < this.replicationQueueNbCapacity; i++) {
171 this.entriesArray[i] = new HLog.Entry();
172 }
173 this.maxRetriesMultiplier =
174 this.conf.getInt("replication.source.maxretriesmultiplier", 10);
175 this.socketTimeoutMultiplier = maxRetriesMultiplier * maxRetriesMultiplier;
176 this.queue =
177 new PriorityBlockingQueue<Path>(
178 conf.getInt("hbase.regionserver.maxlogs", 32),
179 new LogsComparator());
180 this.conn = HConnectionManager.getConnection(conf);
181 this.zkHelper = manager.getRepZkWrapper();
182 this.ratio = this.conf.getFloat("replication.source.ratio", 0.1f);
183 this.currentPeers = new ArrayList<ServerName>();
184 this.random = new Random();
185 this.replicating = replicating;
186 this.manager = manager;
187 this.sleepForRetries =
188 this.conf.getLong("replication.source.sleepforretries", 1000);
189 this.fs = fs;
190 this.metrics = new ReplicationSourceMetrics(peerClusterZnode);
191 this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf);
192 try {
193 this.clusterId = zkHelper.getUUIDForCluster(zkHelper.getZookeeperWatcher());
194 } catch (KeeperException ke) {
195 throw new IOException("Could not read cluster id", ke);
196 }
197
198
199 this.checkIfQueueRecovered(peerClusterZnode);
200 }
201
202
203
204
205
206 void checkIfQueueRecovered(String peerClusterZnode) {
207 String[] parts = peerClusterZnode.split("-", 2);
208 this.queueRecovered = parts.length != 1;
209 this.peerId = this.queueRecovered ?
210 parts[0] : peerClusterZnode;
211 this.peerClusterZnode = peerClusterZnode;
212
213 if (parts.length < 2) {
214
215 return;
216 }
217
218
219 extractDeadServersFromZNodeString(parts[1], this.deadRegionServers);
220 }
221
222
223
224
225 List<String> getDeadRegionServers() {
226 return Collections.unmodifiableList(this.deadRegionServers);
227 }
228
229
230
231
232
233
234 private static void
235 extractDeadServersFromZNodeString(String deadServerListStr, List<String> result) {
236
237 if (deadServerListStr == null || result == null || deadServerListStr.isEmpty()) return;
238
239
240 int seenCommaCnt = 0;
241 int startIndex = 0;
242 int len = deadServerListStr.length();
243
244 for (int i = 0; i < len; i++) {
245 switch (deadServerListStr.charAt(i)) {
246 case ',':
247 seenCommaCnt += 1;
248 break;
249 case '-':
250 if (seenCommaCnt >= 2) {
251 if (i > startIndex) {
252 result.add(deadServerListStr.substring(startIndex, i));
253 startIndex = i + 1;
254 }
255 seenCommaCnt = 0;
256 }
257 break;
258 default:
259 break;
260 }
261 }
262
263
264 if (startIndex < len - 1) {
265 result.add(deadServerListStr.substring(startIndex, len));
266 }
267
268 LOG.debug("Found dead servers:" + result);
269 }
270
271
272
273
274 private void chooseSinks() {
275 this.currentPeers.clear();
276 List<ServerName> addresses = this.zkHelper.getSlavesAddresses(peerId);
277 Set<ServerName> setOfAddr = new HashSet<ServerName>();
278 int nbPeers = (int) (Math.ceil(addresses.size() * ratio));
279 LOG.info("Getting " + nbPeers +
280 " rs from peer cluster # " + peerId);
281 for (int i = 0; i < nbPeers; i++) {
282 ServerName sn;
283
284 do {
285 sn = addresses.get(this.random.nextInt(addresses.size()));
286 } while (setOfAddr.contains(sn));
287 LOG.info("Choosing peer " + sn);
288 setOfAddr.add(sn);
289 }
290 this.currentPeers.addAll(setOfAddr);
291 }
292
293 @Override
294 public void enqueueLog(Path log) {
295 this.queue.put(log);
296 this.metrics.sizeOfLogQueue.set(queue.size());
297 }
298
299 @Override
300 public void run() {
301 connectToPeers();
302
303 if (!this.isActive()) {
304 return;
305 }
306 int sleepMultiplier = 1;
307
308 while (this.peerClusterId == null) {
309 this.peerClusterId = zkHelper.getPeerUUID(this.peerId);
310 if (this.peerClusterId == null) {
311 if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
312 sleepMultiplier++;
313 }
314 }
315 }
316
317 sleepMultiplier = 1;
318
319 LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
320
321
322
323 if (this.queueRecovered) {
324 try {
325 this.repLogReader.setPosition(this.zkHelper.getHLogRepPosition(
326 this.peerClusterZnode, this.queue.peek().getName()));
327 } catch (KeeperException e) {
328 this.terminate("Couldn't get the position of this recovered queue " +
329 peerClusterZnode, e);
330 }
331 }
332
333 while (isActive()) {
334
335 if (!isPeerEnabled()) {
336 if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
337 sleepMultiplier++;
338 }
339 continue;
340 }
341 Path oldPath = getCurrentPath();
342
343
344
345 boolean hasCurrentPath = getNextPath();
346 if (getCurrentPath() != null && oldPath == null) {
347 sleepMultiplier = 1;
348 }
349 if (!hasCurrentPath) {
350 if (sleepForRetries("No log to process", sleepMultiplier)) {
351 sleepMultiplier++;
352 }
353 continue;
354 }
355 boolean currentWALisBeingWrittenTo = false;
356
357
358
359
360
361
362
363
364 if (!this.queueRecovered && queue.size() == 0) {
365 currentWALisBeingWrittenTo = true;
366 }
367
368 if (!openReader(sleepMultiplier)) {
369
370 sleepMultiplier = 1;
371 continue;
372 }
373
374
375 if (this.reader == null) {
376 if (sleepForRetries("Unable to open a reader", sleepMultiplier)) {
377 sleepMultiplier++;
378 }
379 continue;
380 }
381
382 boolean gotIOE = false;
383 currentNbOperations = 0;
384 currentNbEntries = 0;
385 currentSize = 0;
386 try {
387 if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo)) {
388 continue;
389 }
390 } catch (IOException ioe) {
391 LOG.warn(peerClusterZnode + " Got: ", ioe);
392 gotIOE = true;
393 if (ioe.getCause() instanceof EOFException) {
394
395 boolean considerDumping = false;
396 if (this.queueRecovered) {
397 try {
398 FileStatus stat = this.fs.getFileStatus(this.currentPath);
399 if (stat.getLen() == 0) {
400 LOG.warn(peerClusterZnode + " Got EOF and the file was empty");
401 }
402 considerDumping = true;
403 } catch (IOException e) {
404 LOG.warn(peerClusterZnode + " Got while getting file size: ", e);
405 }
406 } else if (currentNbEntries != 0) {
407 LOG.warn(peerClusterZnode + " Got EOF while reading, " +
408 "looks like this file is broken? " + currentPath);
409 considerDumping = true;
410 currentNbEntries = 0;
411 }
412
413 if (considerDumping &&
414 sleepMultiplier == this.maxRetriesMultiplier &&
415 processEndOfFile()) {
416 continue;
417 }
418 }
419 } finally {
420 try {
421 this.reader = null;
422 this.repLogReader.closeReader();
423 } catch (IOException e) {
424 gotIOE = true;
425 LOG.warn("Unable to finalize the tailing of a file", e);
426 }
427 }
428
429
430
431
432 if (this.isActive() && (gotIOE || currentNbEntries == 0)) {
433 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
434 this.manager.logPositionAndCleanOldLogs(this.currentPath,
435 this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo);
436 this.lastLoggedPosition = this.repLogReader.getPosition();
437 }
438 if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
439 sleepMultiplier++;
440 }
441 continue;
442 }
443 sleepMultiplier = 1;
444 shipEdits(currentWALisBeingWrittenTo);
445
446 }
447 if (this.conn != null) {
448 try {
449 this.conn.close();
450 } catch (IOException e) {
451 LOG.debug("Attempt to close connection failed", e);
452 }
453 }
454 LOG.debug("Source exiting " + peerId);
455 }
456
457
458
459
460
461
462
463
464
465 protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo)
466 throws IOException{
467 long seenEntries = 0;
468 this.repLogReader.seek();
469 HLog.Entry entry =
470 this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
471 while (entry != null) {
472 WALEdit edit = entry.getEdit();
473 this.metrics.logEditsReadRate.inc(1);
474 seenEntries++;
475
476 HLogKey logKey = entry.getKey();
477
478 if (!logKey.getClusterId().equals(peerClusterId)) {
479 removeNonReplicableEdits(edit);
480
481
482 if (!(Bytes.equals(logKey.getTablename(), HConstants.ROOT_TABLE_NAME) ||
483 Bytes.equals(logKey.getTablename(), HConstants.META_TABLE_NAME)) &&
484 edit.size() != 0 && replicating.get()) {
485
486
487
488
489 if (HConstants.DEFAULT_CLUSTER_ID == logKey.getClusterId()) {
490 logKey.setClusterId(this.clusterId);
491 }
492 currentNbOperations += countDistinctRowKeys(edit);
493 currentNbEntries++;
494 currentSize += entry.getEdit().heapSize();
495 } else {
496 this.metrics.logEditsFilteredRate.inc(1);
497 }
498 }
499
500 if (currentSize >= this.replicationQueueSizeCapacity ||
501 currentNbEntries >= this.replicationQueueNbCapacity) {
502 break;
503 }
504 try {
505 entry = this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
506 } catch (IOException ie) {
507 LOG.debug("Break on IOE: " + ie.getMessage());
508 break;
509 }
510 }
511 LOG.debug("currentNbOperations:" + currentNbOperations +
512 " and seenEntries:" + seenEntries +
513 " and size: " + this.currentSize);
514 if (currentWALisBeingWrittenTo) {
515 return false;
516 }
517
518
519 return seenEntries == 0 && processEndOfFile();
520 }
521
522 private void connectToPeers() {
523
524 while (this.isActive() && this.currentPeers.size() == 0) {
525
526 try {
527 chooseSinks();
528 Thread.sleep(this.sleepForRetries);
529 } catch (InterruptedException e) {
530 LOG.error("Interrupted while trying to connect to sinks", e);
531 }
532 }
533 }
534
535
536
537
538
539 protected boolean getNextPath() {
540 try {
541 if (this.currentPath == null) {
542 this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
543 this.metrics.sizeOfLogQueue.set(queue.size());
544 }
545 } catch (InterruptedException e) {
546 LOG.warn("Interrupted while reading edits", e);
547 }
548 return this.currentPath != null;
549 }
550
551
552
553
554
555
556
557 protected boolean openReader(int sleepMultiplier) {
558 try {
559 LOG.debug("Opening log for replication " + this.currentPath.getName() +
560 " at " + this.repLogReader.getPosition());
561 try {
562 this.reader = repLogReader.openReader(this.currentPath);
563 } catch (FileNotFoundException fnfe) {
564 if (this.queueRecovered) {
565
566
567
568 LOG.info("NB dead servers : " + deadRegionServers.size());
569 for (String curDeadServerName : deadRegionServers) {
570 Path deadRsDirectory =
571 new Path(manager.getLogDir().getParent(), curDeadServerName);
572 Path[] locs = new Path[] {
573 new Path(deadRsDirectory, currentPath.getName()),
574 new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT),
575 currentPath.getName()),
576 };
577 for (Path possibleLogLocation : locs) {
578 LOG.info("Possible location " + possibleLogLocation.toUri().toString());
579 if (this.manager.getFs().exists(possibleLogLocation)) {
580
581 LOG.info("Log " + this.currentPath + " still exists at " +
582 possibleLogLocation);
583
584 return true;
585 }
586 }
587 }
588
589
590
591
592
593
594
595 throw new IOException("File from recovered queue is " +
596 "nowhere to be found", fnfe);
597 } else {
598
599 Path archivedLogLocation =
600 new Path(manager.getOldLogDir(), currentPath.getName());
601 if (this.manager.getFs().exists(archivedLogLocation)) {
602 currentPath = archivedLogLocation;
603 LOG.info("Log " + this.currentPath + " was moved to " +
604 archivedLogLocation);
605
606 this.openReader(sleepMultiplier);
607
608 }
609
610 }
611 }
612 } catch (IOException ioe) {
613 LOG.warn(peerClusterZnode + " Got: ", ioe);
614 this.reader = null;
615
616
617 if (sleepMultiplier == this.maxRetriesMultiplier) {
618 LOG.warn("Waited too long for this file, considering dumping");
619 return !processEndOfFile();
620 }
621 }
622 return true;
623 }
624
625
626
627
628
629
630
631 protected boolean sleepForRetries(String msg, int sleepMultiplier) {
632 try {
633 LOG.debug(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
634 Thread.sleep(this.sleepForRetries * sleepMultiplier);
635 } catch (InterruptedException e) {
636 LOG.debug("Interrupted while sleeping between retries");
637 }
638 return sleepMultiplier < maxRetriesMultiplier;
639 }
640
641
642
643
644
645 protected void removeNonReplicableEdits(WALEdit edit) {
646 NavigableMap<byte[], Integer> scopes = edit.getScopes();
647 List<KeyValue> kvs = edit.getKeyValues();
648 for (int i = edit.size()-1; i >= 0; i--) {
649 KeyValue kv = kvs.get(i);
650
651
652 if (scopes == null || !scopes.containsKey(kv.getFamily())) {
653 kvs.remove(i);
654 }
655 }
656 }
657
658
659
660
661
662
663
664 private int countDistinctRowKeys(WALEdit edit) {
665 List<KeyValue> kvs = edit.getKeyValues();
666 int distinctRowKeys = 1;
667 KeyValue lastKV = kvs.get(0);
668 for (int i = 0; i < edit.size(); i++) {
669 if (!kvs.get(i).matchingRow(lastKV)) {
670 distinctRowKeys++;
671 }
672 }
673 return distinctRowKeys;
674 }
675
676
677
678
679
680
681 protected void shipEdits(boolean currentWALisBeingWrittenTo) {
682 int sleepMultiplier = 1;
683 if (this.currentNbEntries == 0) {
684 LOG.warn("Was given 0 edits to ship");
685 return;
686 }
687 while (this.isActive()) {
688 if (!isPeerEnabled()) {
689 if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
690 sleepMultiplier++;
691 }
692 continue;
693 }
694 try {
695 HRegionInterface rrs = getRS();
696 LOG.debug("Replicating " + currentNbEntries);
697 rrs.replicateLogEntries(Arrays.copyOf(this.entriesArray, currentNbEntries));
698 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
699 this.manager.logPositionAndCleanOldLogs(this.currentPath,
700 this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo);
701 this.lastLoggedPosition = this.repLogReader.getPosition();
702 }
703 this.totalReplicatedEdits += currentNbEntries;
704 this.metrics.shippedBatchesRate.inc(1);
705 this.metrics.shippedOpsRate.inc(
706 this.currentNbOperations);
707 this.metrics.setAgeOfLastShippedOp(
708 this.entriesArray[currentNbEntries-1].getKey().getWriteTime());
709 LOG.debug("Replicated in total: " + this.totalReplicatedEdits);
710 break;
711
712 } catch (IOException ioe) {
713
714 this.metrics.refreshAgeOfLastShippedOp();
715 if (ioe instanceof RemoteException) {
716 ioe = ((RemoteException) ioe).unwrapRemoteException();
717 LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe);
718 } else {
719 if (ioe instanceof SocketTimeoutException) {
720
721
722
723 sleepForRetries("Encountered a SocketTimeoutException. Since the " +
724 "call to the remote cluster timed out, which is usually " +
725 "caused by a machine failure or a massive slowdown",
726 this.socketTimeoutMultiplier);
727 } else if (ioe instanceof ConnectException) {
728 LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
729 chooseSinks();
730 } else {
731 LOG.warn("Can't replicate because of a local or network error: ", ioe);
732 }
733 }
734
735 try {
736 boolean down;
737
738 do {
739 down = isSlaveDown();
740 if (down) {
741 if (sleepForRetries("Since we are unable to replicate", sleepMultiplier)) {
742 sleepMultiplier++;
743 } else {
744 chooseSinks();
745 }
746 }
747 } while (this.isActive() && down );
748 } catch (InterruptedException e) {
749 LOG.debug("Interrupted while trying to contact the peer cluster");
750 }
751 }
752 }
753 }
754
755
756
757
758
759
760 protected boolean isPeerEnabled() {
761 return this.replicating.get() && this.zkHelper.getPeerEnabled(peerId);
762 }
763
764
765
766
767
768
769
770
771 protected boolean processEndOfFile() {
772 if (this.queue.size() != 0) {
773 this.currentPath = null;
774 this.repLogReader.finishCurrentFile();
775 this.reader = null;
776 return true;
777 } else if (this.queueRecovered) {
778 this.manager.closeRecoveredQueue(this);
779 LOG.info("Finished recovering the queue");
780 this.running = false;
781 return true;
782 }
783 return false;
784 }
785
786 public void startup() {
787 String n = Thread.currentThread().getName();
788 Thread.UncaughtExceptionHandler handler =
789 new Thread.UncaughtExceptionHandler() {
790 public void uncaughtException(final Thread t, final Throwable e) {
791 LOG.error("Unexpected exception in ReplicationSource," +
792 " currentPath=" + currentPath, e);
793 }
794 };
795 Threads.setDaemonThreadRunning(
796 this, n + ".replicationSource," + peerClusterZnode, handler);
797 }
798
799 public void terminate(String reason) {
800 terminate(reason, null);
801 }
802
803 public void terminate(String reason, Exception cause) {
804 if (cause == null) {
805 LOG.info("Closing source "
806 + this.peerClusterZnode + " because: " + reason);
807
808 } else {
809 LOG.error("Closing source " + this.peerClusterZnode
810 + " because an error occurred: " + reason, cause);
811 }
812 this.running = false;
813
814 if (!Thread.currentThread().equals(this)) {
815 Threads.shutdown(this, this.sleepForRetries);
816 }
817 }
818
819
820
821
822
823
824 private HRegionInterface getRS() throws IOException {
825 if (this.currentPeers.size() == 0) {
826 throw new IOException(this.peerClusterZnode + " has 0 region servers");
827 }
828 ServerName address =
829 currentPeers.get(random.nextInt(this.currentPeers.size()));
830 return this.conn.getHRegionConnection(address.getHostname(), address.getPort());
831 }
832
833
834
835
836
837
838 public boolean isSlaveDown() throws InterruptedException {
839 final CountDownLatch latch = new CountDownLatch(1);
840 Thread pingThread = new Thread() {
841 public void run() {
842 try {
843 HRegionInterface rrs = getRS();
844
845 rrs.getHServerInfo();
846 latch.countDown();
847 } catch (IOException ex) {
848 if (ex instanceof RemoteException) {
849 ex = ((RemoteException) ex).unwrapRemoteException();
850 }
851 LOG.info("Slave cluster looks down: " + ex.getMessage());
852 }
853 }
854 };
855 pingThread.start();
856
857 boolean down = ! latch.await(this.sleepForRetries, TimeUnit.MILLISECONDS);
858 pingThread.interrupt();
859 return down;
860 }
861
862 public String getPeerClusterZnode() {
863 return this.peerClusterZnode;
864 }
865
866 public String getPeerClusterId() {
867 return this.peerId;
868 }
869
870 public Path getCurrentPath() {
871 return this.currentPath;
872 }
873
874 private boolean isActive() {
875 return !this.stopper.isStopped() && this.running;
876 }
877
878
879
880
881 public static class LogsComparator implements Comparator<Path> {
882
883 @Override
884 public int compare(Path o1, Path o2) {
885 return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
886 }
887
888 @Override
889 public boolean equals(Object o) {
890 return true;
891 }
892
893
894
895
896
897
898
899 private long getTS(Path p) {
900 String[] parts = p.getName().split("\\.");
901 return Long.parseLong(parts[parts.length-1]);
902 }
903 }
904 }