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 }
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
479 if (!logKey.getClusterId().equals(peerClusterId)) {
480 removeNonReplicableEdits(edit);
481
482
483 if (!(Bytes.equals(logKey.getTablename(), HConstants.ROOT_TABLE_NAME) ||
484 Bytes.equals(logKey.getTablename(), HConstants.META_TABLE_NAME)) &&
485 edit.size() != 0 && replicating.get()) {
486
487
488
489
490 if (HConstants.DEFAULT_CLUSTER_ID == logKey.getClusterId()) {
491 logKey.setClusterId(this.clusterId);
492 }
493 currentNbOperations += countDistinctRowKeys(edit);
494 currentNbEntries++;
495 currentSize += entry.getEdit().heapSize();
496 } else {
497 this.metrics.logEditsFilteredRate.inc(1);
498 }
499 }
500
501 if (currentSize >= this.replicationQueueSizeCapacity ||
502 currentNbEntries >= this.replicationQueueNbCapacity) {
503 break;
504 }
505 try {
506 entry = this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
507 } catch (IOException ie) {
508 LOG.debug("Break on IOE: " + ie.getMessage());
509 break;
510 }
511 }
512 LOG.debug("currentNbOperations:" + currentNbOperations +
513 " and seenEntries:" + seenEntries +
514 " and size: " + this.currentSize);
515 if (currentWALisBeingWrittenTo) {
516 return false;
517 }
518
519
520 return seenEntries == 0 && processEndOfFile();
521 }
522
523 private void connectToPeers() {
524 int sleepMultiplier = 1;
525
526
527 while (this.isActive() && this.currentPeers.size() == 0) {
528
529 chooseSinks();
530 if (this.isActive() && this.currentPeers.size() == 0) {
531 if (sleepForRetries("Waiting for peers", sleepMultiplier)) {
532 sleepMultiplier++;
533 }
534 }
535 }
536 }
537
538
539
540
541
542 protected boolean getNextPath() {
543 try {
544 if (this.currentPath == null) {
545 this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
546 this.metrics.sizeOfLogQueue.set(queue.size());
547 }
548 } catch (InterruptedException e) {
549 LOG.warn("Interrupted while reading edits", e);
550 }
551 return this.currentPath != null;
552 }
553
554
555
556
557
558
559
560 protected boolean openReader(int sleepMultiplier) {
561 try {
562 LOG.debug("Opening log for replication " + this.currentPath.getName() +
563 " at " + this.repLogReader.getPosition());
564 try {
565 this.reader = repLogReader.openReader(this.currentPath);
566 } catch (FileNotFoundException fnfe) {
567 if (this.queueRecovered) {
568
569
570
571 LOG.info("NB dead servers : " + deadRegionServers.size());
572 for (String curDeadServerName : deadRegionServers) {
573 Path deadRsDirectory =
574 new Path(manager.getLogDir().getParent(), curDeadServerName);
575 Path[] locs = new Path[] {
576 new Path(deadRsDirectory, currentPath.getName()),
577 new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT),
578 currentPath.getName()),
579 };
580 for (Path possibleLogLocation : locs) {
581 LOG.info("Possible location " + possibleLogLocation.toUri().toString());
582 if (this.manager.getFs().exists(possibleLogLocation)) {
583
584 LOG.info("Log " + this.currentPath + " still exists at " +
585 possibleLogLocation);
586
587 return true;
588 }
589 }
590 }
591
592
593
594
595
596
597
598 throw new IOException("File from recovered queue is " +
599 "nowhere to be found", fnfe);
600 } else {
601
602 Path archivedLogLocation =
603 new Path(manager.getOldLogDir(), currentPath.getName());
604 if (this.manager.getFs().exists(archivedLogLocation)) {
605 currentPath = archivedLogLocation;
606 LOG.info("Log " + this.currentPath + " was moved to " +
607 archivedLogLocation);
608
609 this.openReader(sleepMultiplier);
610
611 }
612
613 }
614 }
615 } catch (IOException ioe) {
616 if (ioe instanceof EOFException && isCurrentLogEmpty()) return true;
617 LOG.warn(peerClusterZnode + " Got: ", ioe);
618 this.reader = null;
619 if (ioe.getCause() instanceof NullPointerException) {
620
621
622
623 LOG.warn("Got NPE opening reader, will retry.");
624 } else if (sleepMultiplier == this.maxRetriesMultiplier) {
625
626
627 LOG.warn("Waited too long for this file, considering dumping");
628 return !processEndOfFile();
629 }
630 }
631 return true;
632 }
633
634
635
636
637
638
639
640 private boolean isCurrentLogEmpty() {
641 return (this.repLogReader.getPosition() == 0 && !queueRecovered && queue.size() == 0);
642 }
643
644
645
646
647
648
649
650 protected boolean sleepForRetries(String msg, int sleepMultiplier) {
651 try {
652 LOG.debug(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
653 Thread.sleep(this.sleepForRetries * sleepMultiplier);
654 } catch (InterruptedException e) {
655 LOG.debug("Interrupted while sleeping between retries");
656 }
657 return sleepMultiplier < maxRetriesMultiplier;
658 }
659
660
661
662
663
664 protected void removeNonReplicableEdits(WALEdit edit) {
665 NavigableMap<byte[], Integer> scopes = edit.getScopes();
666 List<KeyValue> kvs = edit.getKeyValues();
667 for (int i = edit.size()-1; i >= 0; i--) {
668 KeyValue kv = kvs.get(i);
669
670
671 if (scopes == null || !scopes.containsKey(kv.getFamily())) {
672 kvs.remove(i);
673 }
674 }
675 }
676
677
678
679
680
681
682
683 private int countDistinctRowKeys(WALEdit edit) {
684 List<KeyValue> kvs = edit.getKeyValues();
685 int distinctRowKeys = 1;
686 KeyValue lastKV = kvs.get(0);
687 for (int i = 0; i < edit.size(); i++) {
688 if (!kvs.get(i).matchingRow(lastKV)) {
689 distinctRowKeys++;
690 }
691 }
692 return distinctRowKeys;
693 }
694
695
696
697
698
699
700 protected void shipEdits(boolean currentWALisBeingWrittenTo) {
701 int sleepMultiplier = 1;
702 if (this.currentNbEntries == 0) {
703 LOG.warn("Was given 0 edits to ship");
704 return;
705 }
706 while (this.isActive()) {
707 if (!isPeerEnabled()) {
708 if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
709 sleepMultiplier++;
710 }
711 continue;
712 }
713 try {
714 HRegionInterface rrs = getRS();
715 LOG.debug("Replicating " + currentNbEntries);
716 rrs.replicateLogEntries(Arrays.copyOf(this.entriesArray, currentNbEntries));
717 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
718 this.manager.logPositionAndCleanOldLogs(this.currentPath,
719 this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo);
720 this.lastLoggedPosition = this.repLogReader.getPosition();
721 }
722 this.totalReplicatedEdits += currentNbEntries;
723 this.metrics.shippedBatchesRate.inc(1);
724 this.metrics.shippedOpsRate.inc(
725 this.currentNbOperations);
726 this.metrics.setAgeOfLastShippedOp(
727 this.entriesArray[currentNbEntries-1].getKey().getWriteTime());
728 LOG.debug("Replicated in total: " + this.totalReplicatedEdits);
729 break;
730
731 } catch (IOException ioe) {
732
733 this.metrics.refreshAgeOfLastShippedOp();
734 if (ioe instanceof RemoteException) {
735 ioe = ((RemoteException) ioe).unwrapRemoteException();
736 LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe);
737 if (ioe instanceof TableNotFoundException) {
738 if (sleepForRetries("A table is missing in the peer cluster. "
739 + "Replication cannot proceed without losing data.", sleepMultiplier)) {
740 sleepMultiplier++;
741 }
742 }
743 } else {
744 if (ioe instanceof SocketTimeoutException) {
745
746
747
748 sleepForRetries("Encountered a SocketTimeoutException. Since the " +
749 "call to the remote cluster timed out, which is usually " +
750 "caused by a machine failure or a massive slowdown",
751 this.socketTimeoutMultiplier);
752 } else if (ioe instanceof ConnectException) {
753 LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
754 chooseSinks();
755 } else {
756 LOG.warn("Can't replicate because of a local or network error: ", ioe);
757 }
758 }
759
760 try {
761 boolean down;
762
763 do {
764 down = isSlaveDown();
765 if (down) {
766 if (sleepForRetries("Since we are unable to replicate", sleepMultiplier)) {
767 sleepMultiplier++;
768 } else {
769 chooseSinks();
770 }
771 }
772 } while (this.isActive() && down );
773 } catch (InterruptedException e) {
774 LOG.debug("Interrupted while trying to contact the peer cluster");
775 }
776 }
777 }
778 }
779
780
781
782
783
784
785 protected boolean isPeerEnabled() {
786 return this.replicating.get() && this.zkHelper.getPeerEnabled(peerId);
787 }
788
789
790
791
792
793
794
795
796 protected boolean processEndOfFile() {
797 if (this.queue.size() != 0) {
798 this.currentPath = null;
799 this.repLogReader.finishCurrentFile();
800 this.reader = null;
801 return true;
802 } else if (this.queueRecovered) {
803 this.manager.closeRecoveredQueue(this);
804 LOG.info("Finished recovering the queue");
805 this.running = false;
806 return true;
807 }
808 return false;
809 }
810
811 public void startup() {
812 String n = Thread.currentThread().getName();
813 Thread.UncaughtExceptionHandler handler =
814 new Thread.UncaughtExceptionHandler() {
815 public void uncaughtException(final Thread t, final Throwable e) {
816 LOG.error("Unexpected exception in ReplicationSource," +
817 " currentPath=" + currentPath, e);
818 }
819 };
820 Threads.setDaemonThreadRunning(
821 this, n + ".replicationSource," + peerClusterZnode, handler);
822 }
823
824 public void terminate(String reason) {
825 terminate(reason, null);
826 }
827
828 public void terminate(String reason, Exception cause) {
829 if (cause == null) {
830 LOG.info("Closing source "
831 + this.peerClusterZnode + " because: " + reason);
832
833 } else {
834 LOG.error("Closing source " + this.peerClusterZnode
835 + " because an error occurred: " + reason, cause);
836 }
837 this.running = false;
838
839 if (!Thread.currentThread().equals(this)) {
840 Threads.shutdown(this, this.sleepForRetries);
841 }
842 }
843
844
845
846
847
848
849 private HRegionInterface getRS() throws IOException {
850 if (this.currentPeers.size() == 0) {
851 throw new IOException(this.peerClusterZnode + " has 0 region servers");
852 }
853 ServerName address =
854 currentPeers.get(random.nextInt(this.currentPeers.size()));
855 return this.conn.getHRegionConnection(address.getHostname(), address.getPort());
856 }
857
858
859
860
861
862
863 public boolean isSlaveDown() throws InterruptedException {
864 final CountDownLatch latch = new CountDownLatch(1);
865 Thread pingThread = new Thread() {
866 public void run() {
867 try {
868 HRegionInterface rrs = getRS();
869
870 rrs.getHServerInfo();
871 latch.countDown();
872 } catch (IOException ex) {
873 if (ex instanceof RemoteException) {
874 ex = ((RemoteException) ex).unwrapRemoteException();
875 }
876 LOG.info("Slave cluster looks down: " + ex.getMessage());
877 }
878 }
879 };
880 pingThread.start();
881
882 boolean down = ! latch.await(this.sleepForRetries, TimeUnit.MILLISECONDS);
883 pingThread.interrupt();
884 return down;
885 }
886
887 public String getPeerClusterZnode() {
888 return this.peerClusterZnode;
889 }
890
891 public String getPeerClusterId() {
892 return this.peerId;
893 }
894
895 public Path getCurrentPath() {
896 return this.currentPath;
897 }
898
899 private boolean isActive() {
900 return !this.stopper.isStopped() && this.running;
901 }
902
903
904
905
906 public static class LogsComparator implements Comparator<Path> {
907
908 @Override
909 public int compare(Path o1, Path o2) {
910 return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
911 }
912
913 @Override
914 public boolean equals(Object o) {
915 return true;
916 }
917
918
919
920
921
922
923
924 private long getTS(Path p) {
925 String[] parts = p.getName().split("\\.");
926 return Long.parseLong(parts[parts.length-1]);
927 }
928 }
929 }