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