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