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