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