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 com.google.common.collect.Lists;
22 import com.google.common.util.concurrent.ListenableFuture;
23 import com.google.common.util.concurrent.Service;
24 import java.io.IOException;
25 import java.util.ArrayList;
26 import java.util.Collection;
27 import java.util.Comparator;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.TreeMap;
31 import java.util.UUID;
32 import java.util.concurrent.ConcurrentHashMap;
33 import java.util.concurrent.PriorityBlockingQueue;
34 import java.util.concurrent.TimeUnit;
35 import java.util.concurrent.atomic.AtomicLong;
36 import org.apache.commons.lang.StringUtils;
37 import org.apache.commons.logging.Log;
38 import org.apache.commons.logging.LogFactory;
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.Cell;
44 import org.apache.hadoop.hbase.CellUtil;
45 import org.apache.hadoop.hbase.HBaseConfiguration;
46 import org.apache.hadoop.hbase.HConstants;
47 import org.apache.hadoop.hbase.Stoppable;
48 import org.apache.hadoop.hbase.TableName;
49 import org.apache.hadoop.hbase.classification.InterfaceAudience;
50 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
51 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
52 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
53 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
54 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
55 import org.apache.hadoop.hbase.replication.ClusterMarkingEntryFilter;
56 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
57 import org.apache.hadoop.hbase.replication.ReplicationException;
58 import org.apache.hadoop.hbase.replication.ReplicationPeer;
59 import org.apache.hadoop.hbase.replication.ReplicationPeers;
60 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
61 import org.apache.hadoop.hbase.replication.ReplicationQueues;
62 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
63 import org.apache.hadoop.hbase.replication.WALEntryFilter;
64 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReaderThread.WALEntryBatch;
65 import org.apache.hadoop.hbase.util.Bytes;
66 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
67 import org.apache.hadoop.hbase.util.FSUtils;
68 import org.apache.hadoop.hbase.util.Pair;
69 import org.apache.hadoop.hbase.util.Threads;
70 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
71 import org.apache.hadoop.hbase.wal.WAL.Entry;
72
73
74
75
76
77
78
79
80
81
82
83
84
85 @InterfaceAudience.Private
86 public class ReplicationSource extends Thread implements ReplicationSourceInterface {
87
88 private static final Log LOG = LogFactory.getLog(ReplicationSource.class);
89 protected ReplicationSourceLogQueue logQueue;
90
91 private int queueSizePerGroup;
92 private ReplicationQueues replicationQueues;
93 private ReplicationPeers replicationPeers;
94
95 private Configuration conf;
96 private ReplicationQueueInfo replicationQueueInfo;
97
98 private String peerId;
99
100 private ReplicationSourceManager manager;
101
102 private Stoppable stopper;
103
104 private long sleepForRetries;
105 private FileSystem fs;
106
107 private UUID clusterId;
108
109 private UUID peerClusterId;
110
111 private AtomicLong totalReplicatedEdits = new AtomicLong(0);
112
113 private AtomicLong totalReplicatedOperations = new AtomicLong(0);
114
115 private String peerClusterZnode;
116
117 private int maxRetriesMultiplier;
118
119 private volatile boolean sourceRunning = false;
120
121 private volatile boolean startupOngoing = false;
122
123 private MetricsSource metrics;
124
125 private ReplicationEndpoint replicationEndpoint;
126
127 private WALEntryFilter walEntryFilter;
128
129 private ReplicationThrottler throttler;
130 private long defaultBandwidth;
131 private long currentBandwidth;
132 private ConcurrentHashMap<String, ReplicationSourceShipperThread> workerThreads =
133 new ConcurrentHashMap<String, ReplicationSourceShipperThread>();
134
135
136 public enum WorkerState {
137 RUNNING,
138 STOPPED,
139 FINISHED
140 }
141
142 private AtomicLong totalBufferUsed;
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157 @Override
158 public void init(final Configuration conf, final FileSystem fs,
159 final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
160 final ReplicationPeers replicationPeers, final Stoppable stopper,
161 final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
162 final MetricsSource metrics)
163 throws IOException {
164 this.stopper = stopper;
165 this.conf = HBaseConfiguration.create(conf);
166 decorateConf();
167 this.sleepForRetries =
168 this.conf.getLong("replication.source.sleepforretries", 1000);
169 this.maxRetriesMultiplier =
170 this.conf.getInt("replication.source.maxretriesmultiplier", 300);
171 this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32);
172 this.logQueue = new ReplicationSourceLogQueue(conf, metrics);
173 this.replicationQueues = replicationQueues;
174 this.replicationPeers = replicationPeers;
175 this.manager = manager;
176 this.fs = fs;
177 this.metrics = metrics;
178 this.clusterId = clusterId;
179
180 this.peerClusterZnode = peerClusterZnode;
181 this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
182
183 this.peerId = this.replicationQueueInfo.getPeerId();
184 this.replicationEndpoint = replicationEndpoint;
185
186 defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
187 currentBandwidth = getCurrentBandwidth();
188 this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0);
189 this.totalBufferUsed = manager.getTotalBufferUsed();
190 LOG.info("peerClusterZnode=" + peerClusterZnode + ", ReplicationSource : " + peerId
191 + ", currentBandwidth=" + this.currentBandwidth);
192 }
193
194 private void decorateConf() {
195 String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
196 if (StringUtils.isNotEmpty(replicationCodec)) {
197 this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
198 }
199 }
200
201 @Override
202 public void enqueueLog(Path log) {
203 String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(log.getName());
204 boolean queueExists = logQueue.enqueueLog(log, logPrefix);
205 if (!queueExists) {
206 if (this.sourceRunning) {
207
208
209
210 final ReplicationSourceShipperThread worker =
211 new ReplicationSourceShipperThread(logPrefix, logQueue, replicationQueueInfo, this);
212 ReplicationSourceShipperThread extant = workerThreads.putIfAbsent(logPrefix, worker);
213 if (extant != null) {
214 LOG.debug("Someone has beat us to start a worker thread for wal group " + logPrefix);
215 } else {
216 LOG.debug("Starting up worker for wal group " + logPrefix);
217 worker.startup();
218 }
219 }
220 }
221 }
222
223 @InterfaceAudience.Private
224 public Map<String, PriorityBlockingQueue<Path>> getQueues() {
225 return logQueue.getQueues();
226 }
227
228 @Override
229 public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
230 throws ReplicationException {
231 String peerId = peerClusterZnode;
232 if (peerId.contains("-")) {
233
234
235 peerId = peerClusterZnode.split("-")[0];
236 }
237 Map<TableName, List<String>> tableCFMap = replicationPeers.getPeer(peerId).getTableCFs();
238 if (tableCFMap != null) {
239 List<String> tableCfs = tableCFMap.get(tableName);
240 if (tableCFMap.containsKey(tableName)
241 && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
242 this.replicationQueues.addHFileRefs(peerId, pairs);
243 metrics.incrSizeOfHFileRefsQueue(pairs.size());
244 } else {
245 LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
246 + Bytes.toString(family) + " to peer id " + peerId);
247 }
248 } else {
249
250
251 this.replicationQueues.addHFileRefs(peerId, pairs);
252 metrics.incrSizeOfHFileRefsQueue(pairs.size());
253 }
254 }
255
256 private void uninitialize() {
257 LOG.debug("Source exiting " + this.peerId);
258 metrics.clear();
259 if (replicationEndpoint.state() == Service.State.STARTING
260 || replicationEndpoint.state() == Service.State.RUNNING) {
261 replicationEndpoint.stopAndWait();
262 }
263 }
264
265 @Override
266 public void run() {
267
268 this.sourceRunning = true;
269 this.setSourceStartupStatus(true);
270 try {
271
272 Service.State state = replicationEndpoint.start().get();
273 if (state != Service.State.RUNNING) {
274 LOG.warn("ReplicationEndpoint was not started. Exiting");
275 uninitialize();
276 this.setSourceStartupStatus(false);
277 return;
278 }
279 } catch (Exception ex) {
280 LOG.warn("Error starting ReplicationEndpoint, exiting", ex);
281 this.setSourceStartupStatus(false);
282 throw new RuntimeException(ex);
283 }
284
285
286 ArrayList<WALEntryFilter> filters = Lists.newArrayList(
287 (WALEntryFilter)new SystemTableWALEntryFilter());
288 WALEntryFilter filterFromEndpoint = this.replicationEndpoint.getWALEntryfilter();
289 if (filterFromEndpoint != null) {
290 filters.add(filterFromEndpoint);
291 }
292 this.walEntryFilter = new ChainWALEntryFilter(filters);
293
294 int sleepMultiplier = 1;
295
296 while (this.isSourceActive() && this.peerClusterId == null) {
297 this.peerClusterId = replicationEndpoint.getPeerUUID();
298 if (this.isSourceActive() && this.peerClusterId == null) {
299 if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
300 sleepMultiplier++;
301 }
302 }
303 }
304
305 if (!this.isSourceActive()) {
306 this.setSourceStartupStatus(false);
307 return;
308 }
309
310
311
312 if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
313 this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
314 + peerClusterId + " which is not allowed by ReplicationEndpoint:"
315 + replicationEndpoint.getClass().getName(), null, false);
316 this.manager.closeQueue(this);
317 this.setSourceStartupStatus(false);
318 return;
319 }
320 LOG.info("Replicating " + clusterId + " -> " + peerClusterId);
321
322 for (Map.Entry<String, PriorityBlockingQueue<Path>> entry : logQueue.getQueues().entrySet()) {
323 String walGroupId = entry.getKey();
324 PriorityBlockingQueue<Path> queue = entry.getValue();
325 final ReplicationSourceShipperThread worker =
326 new ReplicationSourceShipperThread(walGroupId, logQueue, replicationQueueInfo, this);
327 ReplicationSourceShipperThread extant = workerThreads.putIfAbsent(walGroupId, worker);
328 if (extant != null) {
329 LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);
330 } else {
331 LOG.debug("Starting up worker for wal group " + walGroupId);
332 worker.startup();
333 }
334 }
335 this.setSourceStartupStatus(false);
336 }
337
338 private synchronized void setSourceStartupStatus(boolean initializing) {
339 startupOngoing = initializing;
340 if (initializing) {
341 metrics.incrSourceInitializing();
342 } else {
343 metrics.decrSourceInitializing();
344 }
345 }
346
347
348
349
350
351
352
353 protected boolean sleepForRetries(String msg, int sleepMultiplier) {
354 try {
355 if (LOG.isTraceEnabled()) {
356 LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
357 }
358 Thread.sleep(this.sleepForRetries * sleepMultiplier);
359 } catch (InterruptedException e) {
360 LOG.debug("Interrupted while sleeping between retries");
361 Thread.currentThread().interrupt();
362 }
363 return sleepMultiplier < maxRetriesMultiplier;
364 }
365
366
367
368
369
370
371 protected boolean isPeerEnabled() {
372 return this.replicationPeers.getStatusOfPeer(this.peerId);
373 }
374
375 @Override
376 public void startup() {
377 String n = Thread.currentThread().getName();
378 Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() {
379 @Override
380 public void uncaughtException(final Thread t, final Throwable e) {
381 LOG.error("Unexpected exception in ReplicationSource", e);
382 }
383 };
384 Threads
385 .setDaemonThreadRunning(this, n + ".replicationSource," + this.peerClusterZnode, handler);
386 }
387
388 @Override
389 public void terminate(String reason) {
390 terminate(reason, null);
391 }
392
393 @Override
394 public void terminate(String reason, Exception cause) {
395 terminate(reason, cause, true);
396 }
397
398 public void terminate(String reason, Exception cause, boolean join) {
399 if (cause == null) {
400 LOG.info("Closing source "
401 + this.peerClusterZnode + " because: " + reason);
402
403 } else {
404 LOG.error("Closing source " + this.peerClusterZnode
405 + " because an error occurred: " + reason, cause);
406 }
407 this.sourceRunning = false;
408 Collection<ReplicationSourceShipperThread> workers = workerThreads.values();
409 for (ReplicationSourceShipperThread worker : workers) {
410 worker.setWorkerState(WorkerState.STOPPED);
411 worker.entryReader.interrupt();
412 worker.interrupt();
413 }
414 ListenableFuture<Service.State> future = null;
415 if (this.replicationEndpoint != null) {
416 future = this.replicationEndpoint.stop();
417 }
418 if (join) {
419 for (ReplicationSourceShipperThread worker : workers) {
420 Threads.shutdown(worker, this.sleepForRetries);
421 LOG.info("ReplicationSourceWorker " + worker.getName() + " terminated");
422 }
423 if (future != null) {
424 try {
425 future.get(sleepForRetries * maxRetriesMultiplier, TimeUnit.MILLISECONDS);
426 } catch (Exception e) {
427 LOG.warn("Got exception while waiting for endpoint to shutdown for replication source :"
428 + this.peerClusterZnode,
429 e);
430 }
431 }
432 }
433 }
434
435 @Override
436 public String getPeerClusterZnode() {
437 return this.peerClusterZnode;
438 }
439
440 @Override
441 public String getPeerClusterId() {
442 return this.peerId;
443 }
444
445 @Override
446 public Path getCurrentPath() {
447 for (ReplicationSourceShipperThread worker : workerThreads.values()) {
448 if (worker.getCurrentPath() != null) {
449 return worker.getCurrentPath();
450 }
451 }
452 return null;
453 }
454
455 public Path getLastLoggedPath() {
456 for (ReplicationSourceShipperThread worker : workerThreads.values()) {
457 return worker.getLastLoggedPath();
458 }
459 return null;
460 }
461
462 public long getLastLoggedPosition() {
463 for (ReplicationSourceShipperThread worker : workerThreads.values()) {
464 return worker.getLastLoggedPosition();
465 }
466 return 0;
467 }
468
469 public boolean isSourceActive() {
470 return !this.stopper.isStopped() && this.sourceRunning;
471 }
472
473
474
475
476 public static class LogsComparator implements Comparator<Path> {
477
478 @Override
479 public int compare(Path o1, Path o2) {
480 return Long.compare(getTS(o1), getTS(o2));
481 }
482
483
484
485
486
487
488
489 public static long getTS(Path p) {
490 int tsIndex = p.getName().lastIndexOf('.') + 1;
491 return Long.parseLong(p.getName().substring(tsIndex));
492 }
493 }
494
495 @Override
496 public String getStats() {
497 StringBuilder sb = new StringBuilder();
498 sb.append("Total replicated edits: ").append(totalReplicatedEdits)
499 .append(", current progress: \n");
500 for (Map.Entry<String, ReplicationSourceShipperThread> entry : workerThreads.entrySet()) {
501 String walGroupId = entry.getKey();
502 ReplicationSourceShipperThread worker = entry.getValue();
503 long position = worker.getLastLoggedPosition();
504 Path currentPath = worker.getLastLoggedPath();
505 sb.append("walGroup [").append(walGroupId).append("]: ");
506 if (currentPath != null) {
507 sb.append("currently replicating from: ").append(currentPath).append(" at position: ")
508 .append(position).append("\n");
509 } else {
510 sb.append("no replication ongoing, waiting for new log");
511 }
512 }
513 return sb.toString();
514 }
515
516 @Override
517 public MetricsSource getSourceMetrics() {
518 return this.metrics;
519 }
520
521 private long getCurrentBandwidth() {
522 ReplicationPeer replicationPeer = this.replicationPeers.getPeer(peerId);
523 long peerBandwidth = replicationPeer != null ? replicationPeer.getPeerBandwidth() : 0;
524
525 return peerBandwidth != 0 ? peerBandwidth : defaultBandwidth;
526 }
527
528 @Override
529 public Map<String, ReplicationStatus> getWalGroupStatus() {
530 Map<String, ReplicationStatus> sourceReplicationStatus = new TreeMap<>();
531 long lastTimeStamp, ageOfLastShippedOp, replicationDelay, fileSize;
532 for (ReplicationSourceShipperThread worker : workerThreads.values()) {
533 String walGroupId = worker.getWalGroupId();
534 lastTimeStamp = metrics.getLastTimeStampOfWalGroup(walGroupId);
535 ageOfLastShippedOp = metrics.getAgeOfLastShippedOp(walGroupId);
536 int queueSize = logQueue.getQueueSize(walGroupId);
537 replicationDelay =
538 ReplicationLoad.calculateReplicationDelay(ageOfLastShippedOp, lastTimeStamp, queueSize);
539 Path currentPath = worker.getLastLoggedPath();
540 fileSize = -1;
541 if (currentPath != null) {
542 try {
543 fileSize = fs.getContentSummary(currentPath).getLength();
544 } catch (IOException e) {
545
546
547 }
548 } else {
549 currentPath = new Path("NO_LOGS_IN_QUEUE");
550 LOG.warn("No replication ongoing, waiting for new log");
551 }
552 ReplicationStatus.ReplicationStatusBuilder statusBuilder = ReplicationStatus.newBuilder();
553 statusBuilder.withPeerId(this.getPeerClusterId())
554 .withQueueSize(queueSize)
555 .withWalGroup(walGroupId)
556 .withCurrentPath(currentPath)
557 .withCurrentPosition(worker.getLastLoggedPosition())
558 .withFileSize(fileSize)
559 .withAgeOfLastShippedOp(ageOfLastShippedOp)
560 .withReplicationDelay(replicationDelay);
561 sourceReplicationStatus.put(this.getPeerClusterId() + "=>" + walGroupId,
562 statusBuilder.build());
563 }
564 return sourceReplicationStatus;
565 }
566
567
568
569 public class ReplicationSourceShipperThread extends Thread {
570 ReplicationSourceInterface source;
571 String walGroupId;
572 ReplicationSourceLogQueue logQueue;
573 ReplicationQueueInfo replicationQueueInfo;
574
575 private long lastLoggedPosition = -1;
576
577 private volatile Path lastLoggedPath;
578
579 private WorkerState state;
580 ReplicationSourceWALReaderThread entryReader;
581
582 public ReplicationSourceShipperThread(String walGroupId,
583 ReplicationSourceLogQueue logQueue, ReplicationQueueInfo replicationQueueInfo,
584 ReplicationSourceInterface source) {
585 this.walGroupId = walGroupId;
586 this.logQueue = logQueue;
587 this.replicationQueueInfo = replicationQueueInfo;
588 this.source = source;
589 }
590
591 public String getWalGroupId() {
592 return walGroupId;
593 }
594
595 @Override
596 public void run() {
597 setWorkerState(WorkerState.RUNNING);
598
599 while (isWorkerActive()) {
600 int sleepMultiplier = 1;
601
602 if (!isPeerEnabled()) {
603 if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
604 sleepMultiplier++;
605 }
606 continue;
607 }
608 while (entryReader == null) {
609 if (sleepForRetries("Replication WAL entry reader thread not initialized",
610 sleepMultiplier)) {
611 sleepMultiplier++;
612 }
613 if (sleepMultiplier == maxRetriesMultiplier) {
614 LOG.warn("Replication WAL entry reader thread not initialized");
615 }
616 }
617
618 try {
619 WALEntryBatch entryBatch = entryReader.take();
620 shipEdits(entryBatch);
621 if (!entryBatch.hasMoreEntries()) {
622 LOG.debug("Finished recovering queue for group "
623 + walGroupId + " of peer " + peerClusterZnode);
624 metrics.incrCompletedRecoveryQueue();
625 setWorkerState(WorkerState.FINISHED);
626 }
627 } catch (InterruptedException e) {
628 LOG.trace("Interrupted while waiting for next replication entry batch", e);
629 Thread.currentThread().interrupt();
630 }
631 }
632
633 if (getWorkerState() == WorkerState.FINISHED) {
634
635 synchronized (this) {
636 Threads.sleep(100);
637 boolean allOtherTaskDone = true;
638 for (ReplicationSourceShipperThread worker : workerThreads.values()) {
639 if (!worker.equals(this) && worker.getWorkerState() != WorkerState.FINISHED) {
640 allOtherTaskDone = false;
641 break;
642 }
643 }
644 if (allOtherTaskDone) {
645 manager.closeRecoveredQueue(this.source);
646
647 if (source instanceof ReplicationSource) {
648 ((ReplicationSource) source).replicationEndpoint.stop();
649 }
650 LOG.info("Finished recovering queue " + peerClusterZnode
651 + " with the following stats: " + getStats());
652 }
653 }
654 }
655
656 if (state != WorkerState.FINISHED) {
657 setWorkerState(WorkerState.STOPPED);
658 }
659 }
660
661 private void cleanUpHFileRefs(WALEdit edit) throws IOException {
662 String peerId = peerClusterZnode;
663 if (peerId.contains("-")) {
664
665
666 peerId = peerClusterZnode.split("-")[0];
667 }
668 List<Cell> cells = edit.getCells();
669 int totalCells = cells.size();
670 for (int i = 0; i < totalCells; i++) {
671 Cell cell = cells.get(i);
672 if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
673 BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
674 List<StoreDescriptor> stores = bld.getStoresList();
675 int totalStores = stores.size();
676 for (int j = 0; j < totalStores; j++) {
677 List<String> storeFileList = stores.get(j).getStoreFileList();
678 manager.cleanUpHFileRefs(peerId, storeFileList);
679 metrics.decrSizeOfHFileRefsQueue(storeFileList.size());
680 }
681 }
682 }
683 }
684
685 private void checkBandwidthChangeAndResetThrottler() {
686 long peerBandwidth = getCurrentBandwidth();
687 if (peerBandwidth != currentBandwidth) {
688 currentBandwidth = peerBandwidth;
689 throttler.setBandwidth((double) currentBandwidth / 10.0);
690 LOG.info("ReplicationSource : " + peerId
691 + " bandwidth throttling changed, currentBandWidth=" + currentBandwidth);
692 }
693 }
694
695
696
697
698
699 private int getBatchEntrySizeExcludeBulkLoad(WALEntryBatch entryBatch) {
700 int totalSize = 0;
701 for(Entry entry : entryBatch.getWalEntries()) {
702 totalSize += entryReader.getEntrySizeExcludeBulkLoad(entry);
703 }
704 return totalSize;
705 }
706
707
708
709
710 protected void shipEdits(WALEntryBatch entryBatch) {
711 List<Entry> entries = entryBatch.getWalEntries();
712 long lastReadPosition = entryBatch.getLastWalPosition();
713 lastLoggedPath = entryBatch.getLastWalPath();
714 int sleepMultiplier = 0;
715 if (entries.isEmpty()) {
716 updateLogPosition(lastReadPosition);
717
718
719 metrics.setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(), walGroupId);
720 return;
721 }
722 int currentSize = (int) entryBatch.getHeapSize();
723 int sizeExcludeBulkLoad = getBatchEntrySizeExcludeBulkLoad(entryBatch);
724 while (isWorkerActive()) {
725 try {
726 checkBandwidthChangeAndResetThrottler();
727 if (throttler.isEnabled()) {
728 long sleepTicks = throttler.getNextSleepInterval(sizeExcludeBulkLoad);
729 if (sleepTicks > 0) {
730 try {
731 if (LOG.isTraceEnabled()) {
732 LOG.trace("To sleep " + sleepTicks + "ms for throttling control");
733 }
734 Thread.sleep(sleepTicks);
735 } catch (InterruptedException e) {
736 LOG.debug("Interrupted while sleeping for throttling control");
737 Thread.currentThread().interrupt();
738
739
740 continue;
741 }
742
743 throttler.resetStartTick();
744 }
745 }
746
747
748 ReplicationEndpoint.ReplicateContext replicateContext =
749 new ReplicationEndpoint.ReplicateContext();
750 replicateContext.setEntries(entries).setSize(currentSize);
751 replicateContext.setWalGroupId(walGroupId);
752
753 long startTimeNs = System.nanoTime();
754
755 boolean replicated = replicationEndpoint.replicate(replicateContext);
756 long endTimeNs = System.nanoTime();
757
758 if (!replicated) {
759 continue;
760 } else {
761 sleepMultiplier = Math.max(sleepMultiplier - 1, 0);
762 }
763
764 if (this.lastLoggedPosition != lastReadPosition) {
765
766 int size = entries.size();
767 for (int i = 0; i < size; i++) {
768 cleanUpHFileRefs(entries.get(i).getEdit());
769 }
770
771 updateLogPosition(lastReadPosition);
772 }
773 if (throttler.isEnabled()) {
774 throttler.addPushSize(sizeExcludeBulkLoad);
775 }
776 releaseBufferQuota(sizeExcludeBulkLoad);
777 totalReplicatedEdits.addAndGet(entries.size());
778 totalReplicatedOperations.addAndGet(entryBatch.getNbOperations());
779
780 metrics.shipBatch(entryBatch.getNbOperations(), currentSize, entryBatch.getNbHFiles());
781 metrics.setAgeOfLastShippedOp(entries.get(entries.size() - 1).getKey().getWriteTime(),
782 walGroupId);
783 source.getSourceMetrics().updateTableLevelMetrics(entryBatch.getWalEntriesWithSize());
784 if (LOG.isTraceEnabled()) {
785 LOG.trace("Replicated " + totalReplicatedEdits + " entries in total, or "
786 + totalReplicatedOperations + " operations in "
787 + ((endTimeNs - startTimeNs) / 1000000) + " ms");
788 }
789 break;
790 } catch (Exception ex) {
791 LOG.warn(replicationEndpoint.getClass().getName() + " threw unknown exception:"
792 + org.apache.hadoop.util.StringUtils.stringifyException(ex));
793 if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) {
794 sleepMultiplier++;
795 }
796 }
797 }
798 }
799
800 private void updateLogPosition(long lastReadPosition) {
801 manager.logPositionAndCleanOldLogs(lastLoggedPath, peerClusterZnode, lastReadPosition,
802 this.replicationQueueInfo.isQueueRecovered(), false);
803 lastLoggedPosition = lastReadPosition;
804 }
805
806 public void startup() {
807 String n = Thread.currentThread().getName();
808 Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() {
809 @Override
810 public void uncaughtException(final Thread t, final Throwable e) {
811 RSRpcServices.exitIfOOME(e);
812 LOG.error("Unexpected exception in ReplicationSourceWorkerThread," + " currentPath="
813 + getLastLoggedPath(), e);
814 stopper.stop("Unexpected exception in ReplicationSourceWorkerThread");
815 }
816 };
817 Threads.setDaemonThreadRunning(this, n + ".replicationSource." + walGroupId + ","
818 + peerClusterZnode, handler);
819 workerThreads.put(walGroupId, this);
820
821 long startPosition = 0;
822
823 if (this.replicationQueueInfo.isQueueRecovered()) {
824 startPosition = getRecoveredQueueStartPos(startPosition);
825 int numRetries = 0;
826 while (numRetries <= maxRetriesMultiplier) {
827 try {
828 locateRecoveredPaths();
829 break;
830 } catch (IOException e) {
831 LOG.error("Error while locating recovered queue paths, attempt #" + numRetries);
832 numRetries++;
833 }
834 }
835 }
836
837 startWALReaderThread(n, handler, startPosition);
838 }
839
840
841
842 private long getRecoveredQueueStartPos(long startPosition) {
843 try {
844 startPosition = (replicationQueues.getLogPosition(peerClusterZnode,
845 this.logQueue.getQueue(walGroupId).peek().getName()));
846 if (LOG.isTraceEnabled()) {
847 LOG.trace("Recovered queue started with log " +
848 this.logQueue.getQueue(walGroupId).peek() + " at position " + startPosition);
849 }
850 } catch (ReplicationException e) {
851 terminate("Couldn't get the position of this recovered queue " + peerClusterZnode, e);
852 }
853 return startPosition;
854 }
855
856
857 private void startWALReaderThread(String threadName, Thread.UncaughtExceptionHandler handler,
858 long startPosition) {
859 ArrayList<WALEntryFilter> filters = Lists.newArrayList(walEntryFilter,
860 new ClusterMarkingEntryFilter(clusterId, peerClusterId, replicationEndpoint));
861 ChainWALEntryFilter readerFilter = new ChainWALEntryFilter(filters);
862 entryReader = new ReplicationSourceWALReaderThread(manager, replicationQueueInfo, logQueue,
863 startPosition, fs, conf, readerFilter, metrics, ReplicationSource.this,
864 this.walGroupId);
865 Threads.setDaemonThreadRunning(entryReader, threadName
866 + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + peerClusterZnode,
867 handler);
868 }
869
870
871
872 private void locateRecoveredPaths() throws IOException {
873 boolean hasPathChanged = false;
874 PriorityBlockingQueue<Path> newPaths =
875 new PriorityBlockingQueue<Path>(queueSizePerGroup, new LogsComparator());
876 PriorityBlockingQueue<Path> queue = logQueue.getQueue(walGroupId);
877 pathsLoop: for (Path path : queue) {
878 if (fs.exists(path)) {
879 newPaths.add(path);
880 continue;
881 }
882
883 hasPathChanged = true;
884 if (stopper instanceof ReplicationSyncUp.DummyServer) {
885
886
887 Path newPath = getReplSyncUpPath(path);
888 newPaths.add(newPath);
889 continue;
890 } else {
891
892
893 List<String> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
894 LOG.info("NB dead servers : " + deadRegionServers.size());
895 final Path walDir = FSUtils.getWALRootDir(conf);
896 for (String curDeadServerName : deadRegionServers) {
897 final Path deadRsDirectory =
898 new Path(walDir, DefaultWALProvider.getWALDirectoryName(curDeadServerName));
899 Path[] locs = new Path[] { new Path(deadRsDirectory, path.getName()), new Path(
900 deadRsDirectory.suffix(DefaultWALProvider.SPLITTING_EXT), path.getName()) };
901 for (Path possibleLogLocation : locs) {
902 LOG.info("Possible location " + possibleLogLocation.toUri().toString());
903 if (manager.getFs().exists(possibleLogLocation)) {
904
905 LOG.info("Log " + path + " still exists at " + possibleLogLocation);
906 newPaths.add(possibleLogLocation);
907 continue pathsLoop;
908 }
909 }
910 }
911
912 LOG.error(
913 String.format("WAL Path %s doesn't exist and couldn't find its new location", path));
914 newPaths.add(path);
915 }
916 }
917
918 if (hasPathChanged) {
919 if (newPaths.size() != queue.size()) {
920 LOG.error("Recovery queue size is incorrect");
921 throw new IOException("Recovery queue size error");
922 }
923
924
925
926 logQueue.clear(walGroupId);
927 for (Path path : newPaths) {
928 logQueue.enqueueLog(path, walGroupId);
929 }
930 }
931 }
932
933
934
935 private Path getReplSyncUpPath(Path path) throws IOException {
936 FileStatus[] rss = fs.listStatus(manager.getLogDir());
937 for (FileStatus rs : rss) {
938 Path p = rs.getPath();
939 FileStatus[] logs = fs.listStatus(p);
940 for (FileStatus log : logs) {
941 String logName = log.getPath().getName();
942 if (logName.equals(path.getName())) {
943 p = new Path(p, log.getPath().getName());
944 LOG.info("Log " + p.getName() + " found at " + p);
945 return p;
946 }
947 }
948 }
949 LOG.error("Didn't find path for: " + path.getName());
950 return path;
951 }
952
953 public Path getCurrentPath() {
954 return this.entryReader.getCurrentPath();
955 }
956
957 public Path getLastLoggedPath() {
958 return lastLoggedPath;
959 }
960
961 public long getLastLoggedPosition() {
962 return lastLoggedPosition;
963 }
964
965 private boolean isWorkerActive() {
966 return !stopper.isStopped() && state == WorkerState.RUNNING && !isInterrupted();
967 }
968
969 private void terminate(String reason, Exception cause) {
970 if (cause == null) {
971 LOG.info("Closing worker for wal group " + this.walGroupId + " because: " + reason);
972
973 } else {
974 LOG.error("Closing worker for wal group " + this.walGroupId
975 + " because an error occurred: " + reason, cause);
976 }
977 entryReader.interrupt();
978 Threads.shutdown(entryReader, sleepForRetries);
979 setWorkerState(WorkerState.STOPPED);
980 this.interrupt();
981 Threads.shutdown(this, sleepForRetries);
982 LOG.info("ReplicationSourceWorker " + this.getName() + " terminated");
983 }
984
985
986
987
988
989 public void setWorkerState(WorkerState state) {
990 this.state = state;
991 if (entryReader != null) {
992 entryReader.setReaderRunning(state == WorkerState.RUNNING);
993 }
994 }
995
996
997
998
999
1000 public WorkerState getWorkerState() {
1001 return state;
1002 }
1003
1004 private void releaseBufferQuota(int size) {
1005 totalBufferUsed.addAndGet(-size);
1006 }
1007 }
1008 }