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