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.Map;
28 import java.util.UUID;
29 import java.util.concurrent.PriorityBlockingQueue;
30 import java.util.concurrent.TimeUnit;
31
32 import org.apache.commons.lang.StringUtils;
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
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.Cell;
40 import org.apache.hadoop.hbase.CellUtil;
41 import org.apache.hadoop.hbase.HBaseConfiguration;
42 import org.apache.hadoop.hbase.HConstants;
43 import org.apache.hadoop.hbase.Stoppable;
44 import org.apache.hadoop.hbase.TableName;
45 import org.apache.hadoop.hbase.classification.InterfaceAudience;
46 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
47 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
48 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
49 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
50 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
51 import org.apache.hadoop.hbase.replication.ReplicationException;
52 import org.apache.hadoop.hbase.replication.ReplicationPeers;
53 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
54 import org.apache.hadoop.hbase.replication.ReplicationQueues;
55 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
56 import org.apache.hadoop.hbase.replication.WALEntryFilter;
57 import org.apache.hadoop.hbase.util.Bytes;
58 import org.apache.hadoop.hbase.util.CancelableProgressable;
59 import org.apache.hadoop.hbase.util.FSUtils;
60 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
61 import org.apache.hadoop.hbase.util.Threads;
62 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
63 import org.apache.hadoop.hbase.wal.WAL;
64 import org.apache.hadoop.hbase.wal.WALKey;
65
66 import com.google.common.collect.Lists;
67 import com.google.common.util.concurrent.ListenableFuture;
68 import com.google.common.util.concurrent.Service;
69
70
71
72
73
74
75
76
77
78
79
80
81
82 @InterfaceAudience.Private
83 public class ReplicationSource extends Thread
84 implements ReplicationSourceInterface {
85
86 public static final Log LOG = LogFactory.getLog(ReplicationSource.class);
87
88 private PriorityBlockingQueue<Path> queue;
89 private ReplicationQueues replicationQueues;
90 private ReplicationPeers replicationPeers;
91
92 private Configuration conf;
93 private ReplicationQueueInfo replicationQueueInfo;
94
95 private String peerId;
96
97 private ReplicationSourceManager manager;
98
99 private Stoppable stopper;
100
101 private long sleepForRetries;
102
103 private long replicationQueueSizeCapacity;
104
105 private int replicationQueueNbCapacity;
106
107 private WAL.Reader reader;
108
109 private long lastLoggedPosition = -1;
110
111 private volatile Path currentPath;
112 private FileSystem fs;
113
114 private UUID clusterId;
115
116 private UUID peerClusterId;
117
118 private long totalReplicatedEdits = 0;
119
120 private long totalReplicatedOperations = 0;
121
122 private String peerClusterZnode;
123
124 private int maxRetriesMultiplier;
125
126 private int currentNbOperations = 0;
127
128 private int currentSize = 0;
129
130 private long currentNbHFiles = 0;
131
132 private volatile boolean running = true;
133
134 private MetricsSource metrics;
135
136 private ReplicationWALReaderManager repLogReader;
137
138 private int logQueueWarnThreshold;
139
140 private ReplicationEndpoint replicationEndpoint;
141
142 private WALEntryFilter walEntryFilter;
143
144 private ReplicationThrottler throttler;
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159 @Override
160 public void init(final Configuration conf, final FileSystem fs,
161 final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
162 final ReplicationPeers replicationPeers, final Stoppable stopper,
163 final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
164 final MetricsSource metrics)
165 throws IOException {
166 this.stopper = stopper;
167 this.conf = HBaseConfiguration.create(conf);
168 decorateConf();
169 this.replicationQueueSizeCapacity =
170 this.conf.getLong("replication.source.size.capacity", 1024*1024*64);
171 this.replicationQueueNbCapacity =
172 this.conf.getInt("replication.source.nb.capacity", 25000);
173 this.sleepForRetries =
174 this.conf.getLong("replication.source.sleepforretries", 1000);
175 this.maxRetriesMultiplier =
176 this.conf.getInt("replication.source.maxretriesmultiplier", 300);
177 this.queue =
178 new PriorityBlockingQueue<Path>(
179 this.conf.getInt("hbase.regionserver.maxlogs", 32),
180 new LogsComparator());
181 long bandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
182 this.throttler = new ReplicationThrottler((double)bandwidth/10.0);
183 this.replicationQueues = replicationQueues;
184 this.replicationPeers = replicationPeers;
185 this.manager = manager;
186 this.fs = fs;
187 this.metrics = metrics;
188 this.repLogReader = new ReplicationWALReaderManager(this.fs, this.conf);
189 this.clusterId = clusterId;
190
191 this.peerClusterZnode = peerClusterZnode;
192 this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
193
194 this.peerId = this.replicationQueueInfo.getPeerId();
195 this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
196 this.replicationEndpoint = replicationEndpoint;
197 }
198
199 private void decorateConf() {
200 String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
201 if (StringUtils.isNotEmpty(replicationCodec)) {
202 this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
203 }
204 }
205
206 @Override
207 public void enqueueLog(Path log) {
208 this.queue.put(log);
209 int queueSize = queue.size();
210 this.metrics.setSizeOfLogQueue(queueSize);
211
212 if (queueSize > this.logQueueWarnThreshold) {
213 LOG.warn("Queue size: " + queueSize +
214 " exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
215 }
216 }
217
218 @Override
219 public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
220 throws ReplicationException {
221 String peerId = peerClusterZnode;
222 if (peerId.contains("-")) {
223
224
225 peerId = peerClusterZnode.split("-")[0];
226 }
227 Map<TableName, List<String>> tableCFMap = replicationPeers.getPeer(peerId).getTableCFs();
228 if (tableCFMap != null) {
229 List<String> tableCfs = tableCFMap.get(tableName);
230 if (tableCFMap.containsKey(tableName)
231 && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
232 this.replicationQueues.addHFileRefs(peerId, files);
233 metrics.incrSizeOfHFileRefsQueue(files.size());
234 } else {
235 LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
236 + Bytes.toString(family) + " to peer id " + peerId);
237 }
238 } else {
239
240
241 this.replicationQueues.addHFileRefs(peerId, files);
242 metrics.incrSizeOfHFileRefsQueue(files.size());
243 }
244 }
245
246 private void uninitialize() {
247 LOG.debug("Source exiting " + this.peerId);
248 metrics.clear();
249 if (replicationEndpoint.state() == Service.State.STARTING
250 || replicationEndpoint.state() == Service.State.RUNNING) {
251 replicationEndpoint.stopAndWait();
252 }
253 }
254
255 @Override
256 public void run() {
257
258 if (!this.isActive()) {
259 uninitialize();
260 return;
261 }
262
263 try {
264
265 Service.State state = replicationEndpoint.start().get();
266 if (state != Service.State.RUNNING) {
267 LOG.warn("ReplicationEndpoint was not started. Exiting");
268 uninitialize();
269 return;
270 }
271 } catch (Exception ex) {
272 LOG.warn("Error starting ReplicationEndpoint, exiting", ex);
273 throw new RuntimeException(ex);
274 }
275
276
277 ArrayList<WALEntryFilter> filters = Lists.newArrayList(
278 (WALEntryFilter)new SystemTableWALEntryFilter());
279 WALEntryFilter filterFromEndpoint = this.replicationEndpoint.getWALEntryfilter();
280 if (filterFromEndpoint != null) {
281 filters.add(filterFromEndpoint);
282 }
283 this.walEntryFilter = new ChainWALEntryFilter(filters);
284
285 int sleepMultiplier = 1;
286
287 while (this.isActive() && this.peerClusterId == null) {
288 this.peerClusterId = replicationEndpoint.getPeerUUID();
289 if (this.isActive() && this.peerClusterId == null) {
290 if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
291 sleepMultiplier++;
292 }
293 }
294 }
295
296 if (!this.isActive()) {
297 uninitialize();
298 return;
299 }
300
301
302 sleepMultiplier = 1;
303
304
305
306 if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
307 this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
308 + peerClusterId + " which is not allowed by ReplicationEndpoint:"
309 + replicationEndpoint.getClass().getName(), null, false);
310 }
311 LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
312
313
314
315 if (this.replicationQueueInfo.isQueueRecovered()) {
316 try {
317 this.repLogReader.setPosition(this.replicationQueues.getLogPosition(this.peerClusterZnode,
318 this.queue.peek().getName()));
319 if (LOG.isTraceEnabled()) {
320 LOG.trace("Recovered queue started with log " + this.queue.peek() +
321 " at position " + this.repLogReader.getPosition());
322 }
323 } catch (ReplicationException e) {
324 this.terminate("Couldn't get the position of this recovered queue " +
325 this.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.replicationQueueInfo.isQueueRecovered() && 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 currentNbHFiles = 0;
381 List<WAL.Entry> entries = new ArrayList<WAL.Entry>(1);
382 currentSize = 0;
383 try {
384 if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo, entries)) {
385 continue;
386 }
387 } catch (IOException ioe) {
388 LOG.warn(this.peerClusterZnode + " Got: ", ioe);
389 gotIOE = true;
390 if (ioe.getCause() instanceof EOFException) {
391
392 boolean considerDumping = false;
393 if (this.replicationQueueInfo.isQueueRecovered()) {
394 try {
395 FileStatus stat = this.fs.getFileStatus(this.currentPath);
396 if (stat.getLen() == 0) {
397 LOG.warn(this.peerClusterZnode + " Got EOF and the file was empty");
398 }
399 considerDumping = true;
400 } catch (IOException e) {
401 LOG.warn(this.peerClusterZnode + " Got while getting file size: ", e);
402 }
403 }
404
405 if (considerDumping &&
406 sleepMultiplier == this.maxRetriesMultiplier &&
407 processEndOfFile()) {
408 continue;
409 }
410 }
411 } finally {
412 try {
413 this.reader = null;
414 this.repLogReader.closeReader();
415 } catch (IOException e) {
416 gotIOE = true;
417 LOG.warn("Unable to finalize the tailing of a file", e);
418 }
419 }
420
421
422
423
424 if (this.isActive() && (gotIOE || entries.isEmpty())) {
425 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
426 this.manager.logPositionAndCleanOldLogs(this.currentPath,
427 this.peerClusterZnode, this.repLogReader.getPosition(),
428 this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
429 this.lastLoggedPosition = this.repLogReader.getPosition();
430 }
431
432 if (!gotIOE) {
433 sleepMultiplier = 1;
434
435
436 this.metrics.setAgeOfLastShippedOp(System.currentTimeMillis());
437 }
438 if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
439 sleepMultiplier++;
440 }
441 continue;
442 }
443 sleepMultiplier = 1;
444 shipEdits(currentWALisBeingWrittenTo, entries);
445 }
446 uninitialize();
447 }
448
449
450
451
452
453
454
455
456
457
458 protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo,
459 List<WAL.Entry> entries) throws IOException {
460 long seenEntries = 0;
461 if (LOG.isTraceEnabled()) {
462 LOG.trace("Seeking in " + this.currentPath + " at position "
463 + this.repLogReader.getPosition());
464 }
465 this.repLogReader.seek();
466 long positionBeforeRead = this.repLogReader.getPosition();
467 WAL.Entry entry =
468 this.repLogReader.readNextAndSetPosition();
469 while (entry != null) {
470 this.metrics.incrLogEditsRead();
471 seenEntries++;
472
473
474 if (replicationEndpoint.canReplicateToSameCluster()
475 || !entry.getKey().getClusterIds().contains(peerClusterId)) {
476
477 entry = walEntryFilter.filter(entry);
478 WALEdit edit = null;
479 WALKey logKey = null;
480 if (entry != null) {
481 edit = entry.getEdit();
482 logKey = entry.getKey();
483 }
484
485 if (edit != null && edit.size() != 0) {
486
487 logKey.addClusterId(clusterId);
488 currentNbOperations += countDistinctRowKeys(edit);
489 entries.add(entry);
490 currentSize += calculateTotalSizeOfStoreFiles(edit);
491 } else {
492 this.metrics.incrLogEditsFiltered();
493 }
494 }
495
496 if (currentSize >= this.replicationQueueSizeCapacity ||
497 entries.size() >= this.replicationQueueNbCapacity) {
498 break;
499 }
500 try {
501 entry = this.repLogReader.readNextAndSetPosition();
502 } catch (IOException ie) {
503 LOG.debug("Break on IOE: " + ie.getMessage());
504 break;
505 }
506 }
507 metrics.incrLogReadInBytes(this.repLogReader.getPosition() - positionBeforeRead);
508 if (currentWALisBeingWrittenTo) {
509 return false;
510 }
511
512
513 return seenEntries == 0 && processEndOfFile();
514 }
515
516
517
518
519
520
521 private int calculateTotalSizeOfStoreFiles(WALEdit edit) {
522 List<Cell> cells = edit.getCells();
523 int totalStoreFilesSize = 0;
524
525 int totalCells = edit.size();
526 for (int i = 0; i < totalCells; i++) {
527 if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) {
528 try {
529 BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i));
530 List<StoreDescriptor> stores = bld.getStoresList();
531 int totalStores = stores.size();
532 for (int j = 0; j < totalStores; j++) {
533 totalStoreFilesSize += stores.get(j).getStoreFileSize();
534 }
535 } catch (IOException e) {
536 LOG.error("Failed to deserialize bulk load entry from wal edit. "
537 + "Size of HFiles part of cell will not be considered in replication "
538 + "request size calculation.", e);
539 }
540 }
541 }
542 return totalStoreFilesSize;
543 }
544
545 private void cleanUpHFileRefs(WALEdit edit) throws IOException {
546 String peerId = peerClusterZnode;
547 if (peerId.contains("-")) {
548
549
550 peerId = peerClusterZnode.split("-")[0];
551 }
552 List<Cell> cells = edit.getCells();
553 int totalCells = cells.size();
554 for (int i = 0; i < totalCells; i++) {
555 Cell cell = cells.get(i);
556 if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
557 BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
558 List<StoreDescriptor> stores = bld.getStoresList();
559 int totalStores = stores.size();
560 for (int j = 0; j < totalStores; j++) {
561 List<String> storeFileList = stores.get(j).getStoreFileList();
562 manager.cleanUpHFileRefs(peerId, storeFileList);
563 metrics.decrSizeOfHFileRefsQueue(storeFileList.size());
564 }
565 }
566 }
567 }
568
569
570
571
572
573 protected boolean getNextPath() {
574 try {
575 if (this.currentPath == null) {
576 this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
577 this.metrics.setSizeOfLogQueue(queue.size());
578 if (this.currentPath != null) {
579 this.manager.cleanOldLogs(this.currentPath.getName(),
580 this.peerId,
581 this.replicationQueueInfo.isQueueRecovered());
582 if (LOG.isTraceEnabled()) {
583 LOG.trace("New log: " + this.currentPath);
584 }
585 }
586 }
587 } catch (InterruptedException e) {
588 LOG.warn("Interrupted while reading edits", e);
589 }
590 return this.currentPath != null;
591 }
592
593
594
595
596
597
598
599 protected boolean openReader(int sleepMultiplier) {
600 try {
601 try {
602 if (LOG.isTraceEnabled()) {
603 LOG.trace("Opening log " + this.currentPath);
604 }
605 this.reader = repLogReader.openReader(this.currentPath);
606 } catch (FileNotFoundException fnfe) {
607 if (this.replicationQueueInfo.isQueueRecovered()) {
608
609
610
611 List<String> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
612 LOG.info("NB dead servers : " + deadRegionServers.size());
613 final Path rootDir = FSUtils.getRootDir(this.conf);
614 for (String curDeadServerName : deadRegionServers) {
615 final Path deadRsDirectory = new Path(rootDir,
616 DefaultWALProvider.getWALDirectoryName(curDeadServerName));
617 Path[] locs = new Path[] {
618 new Path(deadRsDirectory, currentPath.getName()),
619 new Path(deadRsDirectory.suffix(DefaultWALProvider.SPLITTING_EXT),
620 currentPath.getName()),
621 };
622 for (Path possibleLogLocation : locs) {
623 LOG.info("Possible location " + possibleLogLocation.toUri().toString());
624 if (this.manager.getFs().exists(possibleLogLocation)) {
625
626 LOG.info("Log " + this.currentPath + " still exists at " +
627 possibleLogLocation);
628
629
630 return true;
631 }
632 }
633 }
634
635
636 if (stopper instanceof ReplicationSyncUp.DummyServer) {
637
638
639 FileStatus[] rss = fs.listStatus(manager.getLogDir());
640 for (FileStatus rs : rss) {
641 Path p = rs.getPath();
642 FileStatus[] logs = fs.listStatus(p);
643 for (FileStatus log : logs) {
644 p = new Path(p, log.getPath().getName());
645 if (p.getName().equals(currentPath.getName())) {
646 currentPath = p;
647 LOG.info("Log " + currentPath.getName() + " found at " + currentPath);
648
649 this.openReader(sleepMultiplier);
650 return true;
651 }
652 }
653 }
654 }
655
656
657
658
659
660
661
662
663 throw new IOException("File from recovered queue is " +
664 "nowhere to be found", fnfe);
665 } else {
666
667 Path archivedLogLocation =
668 new Path(manager.getOldLogDir(), currentPath.getName());
669 if (this.manager.getFs().exists(archivedLogLocation)) {
670 currentPath = archivedLogLocation;
671 LOG.info("Log " + this.currentPath + " was moved to " +
672 archivedLogLocation);
673
674 this.openReader(sleepMultiplier);
675
676 }
677
678 }
679 }
680 } catch (LeaseNotRecoveredException lnre) {
681
682 LOG.warn(peerClusterZnode + " Try to recover the WAL lease " + currentPath, lnre);
683 recoverLease(conf, currentPath);
684 this.reader = null;
685 } catch (IOException ioe) {
686 if (ioe instanceof EOFException && isCurrentLogEmpty()) return true;
687 LOG.warn(this.peerClusterZnode + " Got: ", ioe);
688 this.reader = null;
689 if (ioe.getCause() instanceof NullPointerException) {
690
691
692
693 LOG.warn("Got NPE opening reader, will retry.");
694 } else if (sleepMultiplier == this.maxRetriesMultiplier) {
695
696
697 LOG.warn("Waited too long for this file, considering dumping");
698 return !processEndOfFile();
699 }
700 }
701 return true;
702 }
703
704 private void recoverLease(final Configuration conf, final Path path) {
705 try {
706 final FileSystem dfs = FSUtils.getCurrentFileSystem(conf);
707 FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
708 fsUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
709 @Override
710 public boolean progress() {
711 LOG.debug("recover WAL lease: " + path);
712 return isActive();
713 }
714 });
715 } catch (IOException e) {
716 LOG.warn("unable to recover lease for WAL: " + path, e);
717 }
718 }
719
720
721
722
723
724
725
726 private boolean isCurrentLogEmpty() {
727 return (this.repLogReader.getPosition() == 0 &&
728 !this.replicationQueueInfo.isQueueRecovered() && queue.size() == 0);
729 }
730
731
732
733
734
735
736
737 protected boolean sleepForRetries(String msg, int sleepMultiplier) {
738 try {
739 if (LOG.isTraceEnabled()) {
740 LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
741 }
742 Thread.sleep(this.sleepForRetries * sleepMultiplier);
743 } catch (InterruptedException e) {
744 LOG.debug("Interrupted while sleeping between retries");
745 Thread.currentThread().interrupt();
746 }
747 return sleepMultiplier < maxRetriesMultiplier;
748 }
749
750
751
752
753
754
755
756 private int countDistinctRowKeys(WALEdit edit) {
757 List<Cell> cells = edit.getCells();
758 int distinctRowKeys = 1;
759 int totalHFileEntries = 0;
760 Cell lastCell = cells.get(0);
761 int totalCells = edit.size();
762 for (int i = 0; i < totalCells; i++) {
763
764 if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) {
765 try {
766 BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i));
767 List<StoreDescriptor> stores = bld.getStoresList();
768 int totalStores = stores.size();
769 for (int j = 0; j < totalStores; j++) {
770 totalHFileEntries += stores.get(j).getStoreFileList().size();
771 }
772 } catch (IOException e) {
773 LOG.error("Failed to deserialize bulk load entry from wal edit. "
774 + "Then its hfiles count will not be added into metric.");
775 }
776 }
777 if (!CellUtil.matchingRow(cells.get(i), lastCell)) {
778 distinctRowKeys++;
779 }
780 lastCell = cells.get(i);
781 }
782 currentNbHFiles += totalHFileEntries;
783 return distinctRowKeys + totalHFileEntries;
784 }
785
786
787
788
789
790
791 protected void shipEdits(boolean currentWALisBeingWrittenTo, List<WAL.Entry> entries) {
792 int sleepMultiplier = 0;
793 if (entries.isEmpty()) {
794 LOG.warn("Was given 0 edits to ship");
795 return;
796 }
797 while (this.isActive()) {
798 try {
799 if (this.throttler.isEnabled()) {
800 long sleepTicks = this.throttler.getNextSleepInterval(currentSize);
801 if (sleepTicks > 0) {
802 try {
803 if (LOG.isTraceEnabled()) {
804 LOG.trace("To sleep " + sleepTicks + "ms for throttling control");
805 }
806 Thread.sleep(sleepTicks);
807 } catch (InterruptedException e) {
808 LOG.debug("Interrupted while sleeping for throttling control");
809 Thread.currentThread().interrupt();
810
811
812 continue;
813 }
814
815 this.throttler.resetStartTick();
816 }
817 }
818
819 ReplicationEndpoint.ReplicateContext replicateContext = new ReplicationEndpoint.ReplicateContext();
820 replicateContext.setEntries(entries).setSize(currentSize);
821
822 long startTimeNs = System.nanoTime();
823
824 boolean replicated = replicationEndpoint.replicate(replicateContext);
825 long endTimeNs = System.nanoTime();
826
827 if (!replicated) {
828 continue;
829 } else {
830 sleepMultiplier = Math.max(sleepMultiplier-1, 0);
831 }
832
833 if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
834
835 int size = entries.size();
836 for (int i = 0; i < size; i++) {
837 cleanUpHFileRefs(entries.get(i).getEdit());
838 }
839
840 this.manager.logPositionAndCleanOldLogs(this.currentPath,
841 this.peerClusterZnode, this.repLogReader.getPosition(),
842 this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
843 this.lastLoggedPosition = this.repLogReader.getPosition();
844 }
845 if (this.throttler.isEnabled()) {
846 this.throttler.addPushSize(currentSize);
847 }
848 this.totalReplicatedEdits += entries.size();
849 this.totalReplicatedOperations += currentNbOperations;
850 this.metrics.shipBatch(this.currentNbOperations, this.currentSize/1024, currentNbHFiles);
851 this.metrics.setAgeOfLastShippedOp(entries.get(entries.size()-1).getKey().getWriteTime());
852 if (LOG.isTraceEnabled()) {
853 LOG.trace("Replicated " + this.totalReplicatedEdits + " entries in total, or "
854 + this.totalReplicatedOperations + " operations in " +
855 ((endTimeNs - startTimeNs)/1000000) + " ms");
856 }
857 break;
858 } catch (Exception ex) {
859 LOG.warn(replicationEndpoint.getClass().getName() + " threw unknown exception:" +
860 org.apache.hadoop.util.StringUtils.stringifyException(ex));
861 if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) {
862 sleepMultiplier++;
863 }
864 }
865 }
866 }
867
868
869
870
871
872
873 protected boolean isPeerEnabled() {
874 return this.replicationPeers.getStatusOfPeer(this.peerId);
875 }
876
877
878
879
880
881
882
883
884 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
885 justification="Yeah, this is how it works")
886 protected boolean processEndOfFile() {
887 if (this.queue.size() != 0) {
888 if (LOG.isTraceEnabled()) {
889 String filesize = "N/A";
890 try {
891 FileStatus stat = this.fs.getFileStatus(this.currentPath);
892 filesize = stat.getLen()+"";
893 } catch (IOException ex) {}
894 LOG.trace("Reached the end of a log, stats: " + getStats() +
895 ", and the length of the file is " + filesize);
896 }
897 this.currentPath = null;
898 this.repLogReader.finishCurrentFile();
899 this.reader = null;
900 return true;
901 } else if (this.replicationQueueInfo.isQueueRecovered()) {
902 this.manager.closeRecoveredQueue(this);
903 LOG.info("Finished recovering the queue with the following stats " + getStats());
904 this.running = false;
905 return true;
906 }
907 return false;
908 }
909
910 @Override
911 public void startup() {
912 String n = Thread.currentThread().getName();
913 Thread.UncaughtExceptionHandler handler =
914 new Thread.UncaughtExceptionHandler() {
915 @Override
916 public void uncaughtException(final Thread t, final Throwable e) {
917 LOG.error("Unexpected exception in ReplicationSource," +
918 " currentPath=" + currentPath, e);
919 }
920 };
921 Threads.setDaemonThreadRunning(
922 this, n + ".replicationSource," +
923 this.peerClusterZnode, handler);
924 }
925
926 @Override
927 public void terminate(String reason) {
928 terminate(reason, null);
929 }
930
931 @Override
932 public void terminate(String reason, Exception cause) {
933 terminate(reason, cause, true);
934 }
935
936 public void terminate(String reason, Exception cause, boolean join) {
937 if (cause == null) {
938 LOG.info("Closing source "
939 + this.peerClusterZnode + " because: " + reason);
940
941 } else {
942 LOG.error("Closing source " + this.peerClusterZnode
943 + " because an error occurred: " + reason, cause);
944 }
945 this.running = false;
946 this.interrupt();
947 ListenableFuture<Service.State> future = null;
948 if (this.replicationEndpoint != null) {
949 future = this.replicationEndpoint.stop();
950 }
951 if (join) {
952 Threads.shutdown(this, this.sleepForRetries);
953 if (future != null) {
954 try {
955 future.get();
956 } catch (Exception e) {
957 LOG.warn("Got exception:" + e);
958 }
959 }
960 }
961 }
962
963 @Override
964 public String getPeerClusterZnode() {
965 return this.peerClusterZnode;
966 }
967
968 @Override
969 public String getPeerClusterId() {
970 return this.peerId;
971 }
972
973 @Override
974 public Path getCurrentPath() {
975 return this.currentPath;
976 }
977
978 private boolean isActive() {
979 return !this.stopper.isStopped() && this.running && !isInterrupted();
980 }
981
982
983
984
985 public static class LogsComparator implements Comparator<Path> {
986
987 @Override
988 public int compare(Path o1, Path o2) {
989 return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
990 }
991
992
993
994
995
996
997
998 private static long getTS(Path p) {
999 int tsIndex = p.getName().lastIndexOf('.') + 1;
1000 return Long.parseLong(p.getName().substring(tsIndex));
1001 }
1002 }
1003
1004 @Override
1005 public String getStats() {
1006 long position = this.repLogReader.getPosition();
1007 return "Total replicated edits: " + totalReplicatedEdits +
1008 ", currently replicating from: " + this.currentPath +
1009 " at position: " + position;
1010 }
1011
1012
1013
1014
1015
1016 public MetricsSource getSourceMetrics() {
1017 return this.metrics;
1018 }
1019 }