1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication.regionserver;
20
21 import java.io.EOFException;
22 import java.io.FileNotFoundException;
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.Comparator;
26 import java.util.List;
27 import java.util.UUID;
28 import java.util.concurrent.PriorityBlockingQueue;
29 import java.util.concurrent.TimeUnit;
30
31 import org.apache.commons.lang.StringUtils;
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.hbase.classification.InterfaceAudience;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.fs.FileStatus;
37 import org.apache.hadoop.fs.FileSystem;
38 import org.apache.hadoop.fs.Path;
39 import org.apache.hadoop.hbase.HConstants;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.Stoppable;
42 import org.apache.hadoop.hbase.regionserver.wal.HLog;
43 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
44 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
45 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
46 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
47 import org.apache.hadoop.hbase.replication.ReplicationException;
48 import org.apache.hadoop.hbase.replication.ReplicationPeers;
49 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
50 import org.apache.hadoop.hbase.replication.ReplicationQueues;
51 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
52 import org.apache.hadoop.hbase.replication.WALEntryFilter;
53 import org.apache.hadoop.hbase.util.CancelableProgressable;
54 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
55 import org.apache.hadoop.hbase.util.FSUtils;
56 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
57 import org.apache.hadoop.hbase.util.Threads;
58 import com.google.common.collect.Lists;
59 import com.google.common.util.concurrent.ListenableFuture;
60 import com.google.common.util.concurrent.Service;
61
62
63
64
65
66
67
68
69
70
71
72
73
74 @InterfaceAudience.Private
75 public class ReplicationSource extends Thread
76 implements ReplicationSourceInterface {
77
78 public static final Log LOG = LogFactory.getLog(ReplicationSource.class);
79
80 private PriorityBlockingQueue<Path> queue;
81 private ReplicationQueues replicationQueues;
82 private ReplicationPeers replicationPeers;
83
84 private Configuration conf;
85 private ReplicationQueueInfo replicationQueueInfo;
86
87 private String peerId;
88
89 private ReplicationSourceManager manager;
90
91 private Stoppable stopper;
92
93 private long sleepForRetries;
94
95 private long replicationQueueSizeCapacity;
96
97 private int replicationQueueNbCapacity;
98
99 private HLog.Reader reader;
100
101 private long lastLoggedPosition = -1;
102
103 private volatile Path currentPath;
104 private FileSystem fs;
105
106 private UUID clusterId;
107
108 private UUID peerClusterId;
109
110 private long totalReplicatedEdits = 0;
111
112 private long totalReplicatedOperations = 0;
113
114 private String peerClusterZnode;
115
116 private int maxRetriesMultiplier;
117
118 private int currentNbOperations = 0;
119
120 private int currentSize = 0;
121
122 private volatile boolean running = true;
123
124 private MetricsSource metrics;
125
126 private ReplicationHLogReaderManager repLogReader;
127
128 private int logQueueWarnThreshold;
129
130 private ReplicationEndpoint replicationEndpoint;
131
132 private WALEntryFilter walEntryFilter;
133
134 private ReplicationThrottler throttler;
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149 @Override
150 public void init(final Configuration conf, final FileSystem fs,
151 final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
152 final ReplicationPeers replicationPeers, final Stoppable stopper,
153 final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
154 final MetricsSource metrics)
155 throws IOException {
156 this.stopper = stopper;
157 this.conf = conf;
158 decorateConf();
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.queue =
165 new PriorityBlockingQueue<Path>(
166 this.conf.getInt("hbase.regionserver.maxlogs", 32),
167 new LogsComparator());
168 long bandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
169 this.throttler = new ReplicationThrottler((double)bandwidth/10.0);
170 this.replicationQueues = replicationQueues;
171 this.replicationPeers = replicationPeers;
172 this.manager = manager;
173 this.sleepForRetries =
174 this.conf.getLong("replication.source.sleepforretries", 1000);
175 this.fs = fs;
176 this.metrics = metrics;
177 this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf);
178 this.clusterId = clusterId;
179
180 this.peerClusterZnode = peerClusterZnode;
181 this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
182
183 this.peerId = this.replicationQueueInfo.getPeerId();
184 this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
185 this.replicationEndpoint = replicationEndpoint;
186 }
187
188 private void decorateConf() {
189 String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
190 if (StringUtils.isNotEmpty(replicationCodec)) {
191 this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
192 }
193 }
194
195 @Override
196 public void enqueueLog(Path log) {
197 this.queue.put(log);
198 int queueSize = queue.size();
199 this.metrics.setSizeOfLogQueue(queueSize);
200
201 if (queueSize > this.logQueueWarnThreshold) {
202 LOG.warn("Queue size: " + queueSize +
203 " exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
204 }
205 }
206
207 private void uninitialize() {
208 LOG.debug("Source exiting " + this.peerId);
209 metrics.clear();
210 if (replicationEndpoint.state() == Service.State.STARTING
211 || replicationEndpoint.state() == Service.State.RUNNING) {
212 replicationEndpoint.stopAndWait();
213 }
214 }
215
216 @Override
217 public void run() {
218
219 if (!this.isActive()) {
220 uninitialize();
221 return;
222 }
223
224 try {
225
226 Service.State state = replicationEndpoint.start().get();
227 if (state != Service.State.RUNNING) {
228 LOG.warn("ReplicationEndpoint was not started. Exiting");
229 uninitialize();
230 return;
231 }
232 } catch (Exception ex) {
233 LOG.warn("Error starting ReplicationEndpoint, exiting", ex);
234 throw new RuntimeException(ex);
235 }
236
237
238 ArrayList<WALEntryFilter> filters = Lists.newArrayList(
239 (WALEntryFilter)new SystemTableWALEntryFilter());
240 WALEntryFilter filterFromEndpoint = this.replicationEndpoint.getWALEntryfilter();
241 if (filterFromEndpoint != null) {
242 filters.add(filterFromEndpoint);
243 }
244 this.walEntryFilter = new ChainWALEntryFilter(filters);
245
246 int sleepMultiplier = 1;
247
248 while (this.isActive() && this.peerClusterId == null) {
249 this.peerClusterId = replicationEndpoint.getPeerUUID();
250 if (this.isActive() && this.peerClusterId == null) {
251 if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
252 sleepMultiplier++;
253 }
254 }
255 }
256
257 if (!this.isActive()) {
258 uninitialize();
259 return;
260 }
261
262
263 sleepMultiplier = 1;
264
265
266
267 if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
268 this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
269 + peerClusterId + " which is not allowed by ReplicationEndpoint:"
270 + replicationEndpoint.getClass().getName(), null, false);
271 }
272 LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
273
274
275
276 if (this.replicationQueueInfo.isQueueRecovered()) {
277 try {
278 this.repLogReader.setPosition(this.replicationQueues.getLogPosition(this.peerClusterZnode,
279 this.queue.peek().getName()));
280 if (LOG.isTraceEnabled()) {
281 LOG.trace("Recovered queue started with log " + this.queue.peek() +
282 " at position " + this.repLogReader.getPosition());
283 }
284 } catch (ReplicationException e) {
285 this.terminate("Couldn't get the position of this recovered queue " +
286 this.peerClusterZnode, e);
287 }
288 }
289
290 while (isActive()) {
291
292 if (!isPeerEnabled()) {
293 if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
294 sleepMultiplier++;
295 }
296 continue;
297 }
298 Path oldPath = getCurrentPath();
299
300
301
302 boolean hasCurrentPath = getNextPath();
303 if (getCurrentPath() != null && oldPath == null) {
304 sleepMultiplier = 1;
305 }
306 if (!hasCurrentPath) {
307 if (sleepForRetries("No log to process", sleepMultiplier)) {
308 sleepMultiplier++;
309 }
310 continue;
311 }
312 boolean currentWALisBeingWrittenTo = false;
313
314
315
316
317
318
319
320
321 if (!this.replicationQueueInfo.isQueueRecovered() && queue.size() == 0) {
322 currentWALisBeingWrittenTo = true;
323 }
324
325 if (!openReader(sleepMultiplier)) {
326
327 sleepMultiplier = 1;
328 continue;
329 }
330
331
332 if (this.reader == null) {
333 if (sleepForRetries("Unable to open a reader", sleepMultiplier)) {
334 sleepMultiplier++;
335 }
336 continue;
337 }
338
339 boolean gotIOE = false;
340 currentNbOperations = 0;
341 List<HLog.Entry> entries = new ArrayList<HLog.Entry>(1);
342 currentSize = 0;
343 try {
344 if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo, entries)) {
345 continue;
346 }
347 } catch (IOException ioe) {
348 LOG.warn(this.peerClusterZnode + " Got: ", ioe);
349 gotIOE = true;
350 if (ioe.getCause() instanceof EOFException) {
351
352 boolean considerDumping = false;
353 if (this.replicationQueueInfo.isQueueRecovered()) {
354 try {
355 FileStatus stat = this.fs.getFileStatus(this.currentPath);
356 if (stat.getLen() == 0) {
357 LOG.warn(this.peerClusterZnode + " Got EOF and the file was empty");
358 }
359 considerDumping = true;
360 } catch (IOException e) {
361 LOG.warn(this.peerClusterZnode + " Got while getting file size: ", e);
362 }
363 }
364
365 if (considerDumping &&
366 sleepMultiplier == this.maxRetriesMultiplier &&
367 processEndOfFile()) {
368 continue;
369 }
370 }
371 } finally {
372 try {
373 this.reader = null;
374 this.repLogReader.closeReader();
375 } catch (IOException e) {
376 gotIOE = true;
377 LOG.warn("Unable to finalize the tailing of a file", e);
378 }
379 }
380
381
382
383
384 if (this.isActive() && (gotIOE || entries.isEmpty())) {
385 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
386 this.manager.logPositionAndCleanOldLogs(this.currentPath,
387 this.peerClusterZnode, this.repLogReader.getPosition(),
388 this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
389 this.lastLoggedPosition = this.repLogReader.getPosition();
390 }
391
392 if (!gotIOE) {
393 sleepMultiplier = 1;
394
395
396 this.metrics.setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTimeMillis());
397 }
398 if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
399 sleepMultiplier++;
400 }
401 continue;
402 }
403 sleepMultiplier = 1;
404 shipEdits(currentWALisBeingWrittenTo, entries);
405 }
406 uninitialize();
407 }
408
409
410
411
412
413
414
415
416
417
418 protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo,
419 List<HLog.Entry> entries) throws IOException{
420 long seenEntries = 0;
421 if (LOG.isTraceEnabled()) {
422 LOG.trace("Seeking in " + this.currentPath + " at position "
423 + this.repLogReader.getPosition());
424 }
425 this.repLogReader.seek();
426 long positionBeforeRead = this.repLogReader.getPosition();
427 HLog.Entry entry =
428 this.repLogReader.readNextAndSetPosition();
429 while (entry != null) {
430 this.metrics.incrLogEditsRead();
431 seenEntries++;
432
433
434 if (replicationEndpoint.canReplicateToSameCluster()
435 || !entry.getKey().getClusterIds().contains(peerClusterId)) {
436
437 entry = walEntryFilter.filter(entry);
438 WALEdit edit = null;
439 HLogKey logKey = null;
440 if (entry != null) {
441 edit = entry.getEdit();
442 logKey = entry.getKey();
443 }
444
445 if (edit != null && edit.size() != 0) {
446
447 logKey.addClusterId(clusterId);
448 currentNbOperations += countDistinctRowKeys(edit);
449 entries.add(entry);
450 currentSize += entry.getEdit().heapSize();
451 } else {
452 this.metrics.incrLogEditsFiltered();
453 }
454 }
455
456 if (currentSize >= this.replicationQueueSizeCapacity ||
457 entries.size() >= this.replicationQueueNbCapacity) {
458 break;
459 }
460 try {
461 entry = this.repLogReader.readNextAndSetPosition();
462 } catch (IOException ie) {
463 LOG.debug("Break on IOE: " + ie.getMessage());
464 break;
465 }
466 }
467 metrics.incrLogReadInBytes(this.repLogReader.getPosition() - positionBeforeRead);
468 if (currentWALisBeingWrittenTo) {
469 return false;
470 }
471
472
473 return seenEntries == 0 && processEndOfFile();
474 }
475
476
477
478
479
480 protected boolean getNextPath() {
481 try {
482 if (this.currentPath == null) {
483 this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
484 this.metrics.setSizeOfLogQueue(queue.size());
485 if (this.currentPath != null) {
486 this.manager.cleanOldLogs(this.currentPath.getName(),
487 this.peerId,
488 this.replicationQueueInfo.isQueueRecovered());
489 if (LOG.isTraceEnabled()) {
490 LOG.trace("New log: " + this.currentPath);
491 }
492 }
493 }
494 } catch (InterruptedException e) {
495 LOG.warn("Interrupted while reading edits", e);
496 }
497 return this.currentPath != null;
498 }
499
500
501
502
503
504
505
506 protected boolean openReader(int sleepMultiplier) {
507 try {
508 try {
509 if (LOG.isTraceEnabled()) {
510 LOG.trace("Opening log " + this.currentPath);
511 }
512 this.reader = repLogReader.openReader(this.currentPath);
513 } catch (FileNotFoundException fnfe) {
514 if (this.replicationQueueInfo.isQueueRecovered()) {
515
516
517
518 List<String> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
519 LOG.info("NB dead servers : " + deadRegionServers.size());
520 for (String curDeadServerName : deadRegionServers) {
521 Path deadRsDirectory =
522 new Path(manager.getLogDir().getParent(), curDeadServerName);
523 Path[] locs = new Path[] {
524 new Path(deadRsDirectory, currentPath.getName()),
525 new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT),
526 currentPath.getName()),
527 };
528 for (Path possibleLogLocation : locs) {
529 LOG.info("Possible location " + possibleLogLocation.toUri().toString());
530 if (this.manager.getFs().exists(possibleLogLocation)) {
531
532 LOG.info("Log " + this.currentPath + " still exists at " +
533 possibleLogLocation);
534
535 return true;
536 }
537 }
538 }
539
540
541 if (stopper instanceof ReplicationSyncUp.DummyServer) {
542 FileStatus[] rss = fs.listStatus(manager.getLogDir());
543 for (FileStatus rs : rss) {
544 Path p = rs.getPath();
545 FileStatus[] logs = fs.listStatus(p);
546 for (FileStatus log : logs) {
547 p = new Path(p, log.getPath().getName());
548 if (p.getName().equals(currentPath.getName())) {
549 currentPath = p;
550 LOG.info("Log " + this.currentPath + " exists under " + manager.getLogDir());
551
552 this.openReader(sleepMultiplier);
553 return true;
554 }
555 }
556 }
557 }
558
559
560
561
562
563
564
565
566 throw new IOException("File from recovered queue is " +
567 "nowhere to be found", fnfe);
568 } else {
569
570 Path archivedLogLocation =
571 new Path(manager.getOldLogDir(), currentPath.getName());
572 if (this.manager.getFs().exists(archivedLogLocation)) {
573 currentPath = archivedLogLocation;
574 LOG.info("Log " + this.currentPath + " was moved to " +
575 archivedLogLocation);
576
577 this.openReader(sleepMultiplier);
578
579 }
580
581 }
582 }
583 } catch (LeaseNotRecoveredException lnre) {
584
585 LOG.warn(peerClusterZnode + " Try to recover the WAL lease " + currentPath, lnre);
586 recoverLease(conf, currentPath);
587 this.reader = null;
588 } catch (IOException ioe) {
589 if (ioe instanceof EOFException && isCurrentLogEmpty()) return true;
590 LOG.warn(this.peerClusterZnode + " Got: ", ioe);
591 this.reader = null;
592 if (ioe.getCause() instanceof NullPointerException) {
593
594
595
596 LOG.warn("Got NPE opening reader, will retry.");
597 } else if (sleepMultiplier == this.maxRetriesMultiplier) {
598
599
600 LOG.warn("Waited too long for this file, considering dumping");
601 return !processEndOfFile();
602 }
603 }
604 return true;
605 }
606
607 private void recoverLease(final Configuration conf, final Path path) {
608 try {
609 final FileSystem dfs = FSUtils.getCurrentFileSystem(conf);
610 FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
611 fsUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
612 @Override
613 public boolean progress() {
614 LOG.debug("recover WAL lease: " + path);
615 return isActive();
616 }
617 });
618 } catch (IOException e) {
619 LOG.warn("unable to recover lease for WAL: " + path, e);
620 }
621 }
622
623
624
625
626
627
628
629 private boolean isCurrentLogEmpty() {
630 return (this.repLogReader.getPosition() == 0 &&
631 !this.replicationQueueInfo.isQueueRecovered() && queue.size() == 0);
632 }
633
634
635
636
637
638
639
640 protected boolean sleepForRetries(String msg, int sleepMultiplier) {
641 try {
642 if (LOG.isTraceEnabled()) {
643 LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
644 }
645 Thread.sleep(this.sleepForRetries * sleepMultiplier);
646 } catch (InterruptedException e) {
647 LOG.debug("Interrupted while sleeping between retries");
648 Thread.currentThread().interrupt();
649 }
650 return sleepMultiplier < maxRetriesMultiplier;
651 }
652
653
654
655
656
657
658
659 private int countDistinctRowKeys(WALEdit edit) {
660 List<KeyValue> kvs = edit.getKeyValues();
661 int distinctRowKeys = 1;
662 KeyValue lastKV = kvs.get(0);
663 for (int i = 0; i < edit.size(); i++) {
664 if (!kvs.get(i).matchingRow(lastKV)) {
665 distinctRowKeys++;
666 }
667 lastKV = kvs.get(i);
668 }
669 return distinctRowKeys;
670 }
671
672
673
674
675
676
677 protected void shipEdits(boolean currentWALisBeingWrittenTo, List<HLog.Entry> entries) {
678 int sleepMultiplier = 0;
679 if (entries.isEmpty()) {
680 LOG.warn("Was given 0 edits to ship");
681 return;
682 }
683 while (this.isActive()) {
684 try {
685 if (this.throttler.isEnabled()) {
686 long sleepTicks = this.throttler.getNextSleepInterval(currentSize);
687 if (sleepTicks > 0) {
688 try {
689 if (LOG.isTraceEnabled()) {
690 LOG.trace("To sleep " + sleepTicks + "ms for throttling control");
691 }
692 Thread.sleep(sleepTicks);
693 } catch (InterruptedException e) {
694 LOG.debug("Interrupted while sleeping for throttling control");
695 Thread.currentThread().interrupt();
696
697
698 continue;
699 }
700
701 this.throttler.resetStartTick();
702 }
703 }
704
705 ReplicationEndpoint.ReplicateContext replicateContext = new ReplicationEndpoint.ReplicateContext();
706 replicateContext.setEntries(entries).setSize(currentSize);
707
708
709 boolean replicated = replicationEndpoint.replicate(replicateContext);
710
711 if (!replicated) {
712 continue;
713 } else {
714 sleepMultiplier = Math.max(sleepMultiplier-1, 0);
715 }
716
717 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
718 this.manager.logPositionAndCleanOldLogs(this.currentPath,
719 this.peerClusterZnode, this.repLogReader.getPosition(),
720 this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
721 this.lastLoggedPosition = this.repLogReader.getPosition();
722 }
723 if (this.throttler.isEnabled()) {
724 this.throttler.addPushSize(currentSize);
725 }
726 this.totalReplicatedEdits += entries.size();
727 this.totalReplicatedOperations += currentNbOperations;
728 this.metrics.shipBatch(this.currentNbOperations, this.currentSize/1024);
729 this.metrics.setAgeOfLastShippedOp(entries.get(entries.size()-1).getKey().getWriteTime());
730 if (LOG.isTraceEnabled()) {
731 LOG.trace("Replicated " + this.totalReplicatedEdits + " entries in total, or "
732 + this.totalReplicatedOperations + " operations");
733 }
734 break;
735 } catch (Exception ex) {
736 LOG.warn(replicationEndpoint.getClass().getName() + " threw unknown exception:" + ex);
737 if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) {
738 sleepMultiplier++;
739 }
740 }
741 }
742 }
743
744
745
746
747
748
749 protected boolean isPeerEnabled() {
750 return this.replicationPeers.getStatusOfPeer(this.peerId);
751 }
752
753
754
755
756
757
758
759
760 protected boolean processEndOfFile() {
761 if (this.queue.size() != 0) {
762 if (LOG.isTraceEnabled()) {
763 String filesize = "N/A";
764 try {
765 FileStatus stat = this.fs.getFileStatus(this.currentPath);
766 filesize = stat.getLen()+"";
767 } catch (IOException ex) {}
768 LOG.trace("Reached the end of a log, stats: " + getStats() +
769 ", and the length of the file is " + filesize);
770 }
771 this.currentPath = null;
772 this.repLogReader.finishCurrentFile();
773 this.reader = null;
774 return true;
775 } else if (this.replicationQueueInfo.isQueueRecovered()) {
776 this.manager.closeRecoveredQueue(this);
777 LOG.info("Finished recovering the queue with the following stats " + getStats());
778 this.running = false;
779 return true;
780 }
781 return false;
782 }
783
784 @Override
785 public void startup() {
786 String n = Thread.currentThread().getName();
787 Thread.UncaughtExceptionHandler handler =
788 new Thread.UncaughtExceptionHandler() {
789 @Override
790 public void uncaughtException(final Thread t, final Throwable e) {
791 LOG.error("Unexpected exception in ReplicationSource," +
792 " currentPath=" + currentPath, e);
793 }
794 };
795 Threads.setDaemonThreadRunning(
796 this, n + ".replicationSource," +
797 this.peerClusterZnode, handler);
798 }
799
800 @Override
801 public void terminate(String reason) {
802 terminate(reason, null);
803 }
804
805 @Override
806 public void terminate(String reason, Exception cause) {
807 terminate(reason, cause, true);
808 }
809
810 public void terminate(String reason, Exception cause, boolean join) {
811 if (cause == null) {
812 LOG.info("Closing source "
813 + this.peerClusterZnode + " because: " + reason);
814
815 } else {
816 LOG.error("Closing source " + this.peerClusterZnode
817 + " because an error occurred: " + reason, cause);
818 }
819 this.running = false;
820 this.interrupt();
821 ListenableFuture<Service.State> future = null;
822 if (this.replicationEndpoint != null) {
823 future = this.replicationEndpoint.stop();
824 }
825 if (join) {
826 Threads.shutdown(this, this.sleepForRetries);
827 if (future != null) {
828 try {
829 future.get();
830 } catch (Exception e) {
831 LOG.warn("Got exception:" + e);
832 }
833 }
834 }
835 }
836
837 @Override
838 public String getPeerClusterZnode() {
839 return this.peerClusterZnode;
840 }
841
842 @Override
843 public String getPeerClusterId() {
844 return this.peerId;
845 }
846
847 @Override
848 public Path getCurrentPath() {
849 return this.currentPath;
850 }
851
852 private boolean isActive() {
853 return !this.stopper.isStopped() && this.running && !isInterrupted();
854 }
855
856
857
858
859 public static class LogsComparator implements Comparator<Path> {
860
861 @Override
862 public int compare(Path o1, Path o2) {
863 return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
864 }
865
866
867
868
869
870
871
872 private static long getTS(Path p) {
873 int tsIndex = p.getName().lastIndexOf('.') + 1;
874 return Long.parseLong(p.getName().substring(tsIndex));
875 }
876 }
877
878 @Override
879 public String getStats() {
880 long position = this.repLogReader.getPosition();
881 return "Total replicated edits: " + totalReplicatedEdits +
882 ", currently replicating from: " + this.currentPath +
883 " at position: " + position;
884 }
885
886
887
888
889
890 public MetricsSource getSourceMetrics() {
891 return this.metrics;
892 }
893 }