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