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