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