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