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.IOException;
23 import java.util.ArrayList;
24 import java.util.List;
25 import java.util.concurrent.BlockingQueue;
26 import java.util.concurrent.LinkedBlockingQueue;
27 import java.util.concurrent.PriorityBlockingQueue;
28 import java.util.concurrent.TimeUnit;
29 import java.util.concurrent.atomic.AtomicLong;
30
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.fs.FileSystem;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.Cell;
37 import org.apache.hadoop.hbase.CellUtil;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.classification.InterfaceAudience;
40 import org.apache.hadoop.hbase.classification.InterfaceStability;
41 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
42 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
43 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
44 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
45 import org.apache.hadoop.hbase.replication.WALEntryFilter;
46 import org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.WALEntryStreamRuntimeException;
47 import org.apache.hadoop.hbase.util.Pair;
48 import org.apache.hadoop.hbase.util.Threads;
49 import org.apache.hadoop.hbase.wal.WAL.Entry;
50 import org.apache.hadoop.hbase.wal.WALKey;
51
52
53
54
55
56
57 @InterfaceAudience.Private
58 @InterfaceStability.Evolving
59 public class ReplicationSourceWALReaderThread extends Thread {
60 private static final Log LOG = LogFactory.getLog(ReplicationSourceWALReaderThread.class);
61
62 private ReplicationSourceLogQueue logQueue;
63 private FileSystem fs;
64 private Configuration conf;
65 private BlockingQueue<WALEntryBatch> entryBatchQueue;
66
67 private long replicationBatchSizeCapacity;
68
69 private int replicationBatchCountCapacity;
70
71 private long lastReadPosition;
72 private Path lastReadPath;
73 private WALEntryFilter filter;
74 private long sleepForRetries;
75
76 private boolean isReaderRunning = true;
77 private ReplicationQueueInfo replicationQueueInfo;
78 private int maxRetriesMultiplier;
79 private MetricsSource metrics;
80
81 private AtomicLong totalBufferUsed;
82 private long totalBufferQuota;
83 private final String walGroupId;
84
85 private ReplicationSource source;
86 private ReplicationSourceManager manager;
87
88
89
90
91
92
93
94
95
96
97
98
99
100 public ReplicationSourceWALReaderThread(ReplicationSourceManager manager,
101 ReplicationQueueInfo replicationQueueInfo, ReplicationSourceLogQueue logQueue,
102 long startPosition, FileSystem fs, Configuration conf, WALEntryFilter filter,
103 MetricsSource metrics, ReplicationSource source, String walGroupId) {
104 this.replicationQueueInfo = replicationQueueInfo;
105 this.logQueue = logQueue;
106 this.walGroupId = walGroupId;
107 this.lastReadPath = logQueue.getQueue(walGroupId).peek();
108 this.lastReadPosition = startPosition;
109 this.fs = fs;
110 this.conf = conf;
111 this.filter = filter;
112 this.replicationBatchSizeCapacity =
113 this.conf.getLong("replication.source.size.capacity", 1024 * 1024 * 64);
114 this.replicationBatchCountCapacity = this.conf.getInt("replication.source.nb.capacity", 25000);
115
116
117 int batchCount = conf.getInt("replication.source.nb.batches", 1);
118 this.totalBufferUsed = manager.getTotalBufferUsed();
119 this.totalBufferQuota = conf.getLong(HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_KEY,
120 HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_DFAULT);
121 this.sleepForRetries =
122 this.conf.getLong("replication.source.sleepforretries", 1000);
123 this.maxRetriesMultiplier =
124 this.conf.getInt("replication.source.maxretriesmultiplier", 300);
125 this.metrics = metrics;
126 this.entryBatchQueue = new LinkedBlockingQueue<>(batchCount);
127 this.source = source;
128 this.manager = manager;
129 LOG.info("peerClusterZnode=" + replicationQueueInfo.getPeerClusterZnode()
130 + ", ReplicationSourceWALReaderThread : " + replicationQueueInfo.getPeerId()
131 + " inited, replicationBatchSizeCapacity=" + replicationBatchSizeCapacity
132 + ", replicationBatchCountCapacity=" + replicationBatchCountCapacity
133 + ", replicationBatchQueueCapacity=" + batchCount);
134 }
135
136 @Override
137 public void run() {
138 int sleepMultiplier = 1;
139 WALEntryBatch batch = null;
140 WALEntryStream entryStream =
141 new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics, walGroupId);
142 try {
143 while (isReaderRunning()) {
144 try {
145 entryStream = new WALEntryStream(logQueue, fs, conf,
146 lastReadPosition, metrics, walGroupId);
147 while (isReaderRunning()) {
148 if (!source.isPeerEnabled()) {
149 Threads.sleep(sleepForRetries);
150 continue;
151 }
152 if (!checkQuota()) {
153 continue;
154 }
155 batch = new WALEntryBatch(replicationBatchCountCapacity);
156 boolean hasNext = entryStream.hasNext();
157 while (hasNext) {
158 Entry entry = entryStream.next();
159 entry = filterEntry(entry);
160 if (entry != null) {
161 WALEdit edit = entry.getEdit();
162 if (edit != null && !edit.isEmpty()) {
163 long entrySize = getEntrySizeIncludeBulkLoad(entry);
164 long entrySizeExcludeBulkLoad = getEntrySizeExcludeBulkLoad(entry);
165 batch.addEntry(entry, entrySize);
166 updateBatchStats(batch, entry, entryStream.getPosition(), entrySize);
167 boolean totalBufferTooLarge = acquireBufferQuota(entrySizeExcludeBulkLoad);
168
169 if (totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity
170 || batch.getNbEntries() >= replicationBatchCountCapacity) {
171 break;
172 }
173 }
174 }
175 hasNext = entryStream.hasNext();
176 }
177
178
179
180 if (updateBatchAndShippingQueue(entryStream, batch, hasNext, false)) {
181 sleepMultiplier = 1;
182 }
183 }
184 } catch (IOException | WALEntryFilterRetryableException
185 | WALEntryStreamRuntimeException e) {
186 if (handleEofException(e, entryStream, batch)) {
187 sleepMultiplier = 1;
188 } else {
189 if (sleepMultiplier < maxRetriesMultiplier) {
190 LOG.debug("Failed to read stream of replication entries "
191 + "or replication filter is recovering " + e);
192 sleepMultiplier++;
193 } else {
194 LOG.error("Failed to read stream of replication entries "
195 + "or replication filter is recovering " + e);
196 }
197 Threads.sleep(sleepForRetries * sleepMultiplier);
198 }
199 } catch (InterruptedException e) {
200 LOG.trace("Interrupted while sleeping between WAL reads");
201 Thread.currentThread().interrupt();
202 } finally {
203 entryStream.close();
204 }
205 }
206 } catch (IOException e) {
207 if (sleepMultiplier < maxRetriesMultiplier) {
208 LOG.debug("Failed to read stream of replication entries: " + e);
209 sleepMultiplier++;
210 } else {
211 LOG.error("Failed to read stream of replication entries", e);
212 }
213 Threads.sleep(sleepForRetries * sleepMultiplier);
214 } catch (InterruptedException e) {
215 LOG.trace("Interrupted while sleeping between WAL reads");
216 Thread.currentThread().interrupt();
217 }
218 }
219
220
221
222
223
224
225
226
227
228
229
230
231
232 private boolean updateBatchAndShippingQueue(WALEntryStream entryStream, WALEntryBatch batch,
233 boolean hasMoreData, boolean isEOFException) throws InterruptedException, IOException {
234 updateBatch(entryStream, batch, hasMoreData, isEOFException);
235 boolean isDataQueued = false;
236 if (isShippable(batch)) {
237 isDataQueued = true;
238 entryBatchQueue.put(batch);
239 if (!batch.hasMoreEntries()) {
240
241 LOG.debug("Stopping the reader after recovering the queue");
242 setReaderRunning(false);
243 }
244 } else {
245 Thread.sleep(sleepForRetries);
246 }
247
248 if (!isEOFException) {
249 resetStream(entryStream);
250 }
251 return isDataQueued;
252 }
253
254 private void updateBatch(WALEntryStream entryStream, WALEntryBatch batch, boolean moreData,
255 boolean isEOFException) {
256 logMessage(batch);
257
258
259 if (isEOFException) {
260 batch.updatePosition(lastReadPath, lastReadPosition);
261 } else {
262 batch.updatePosition(entryStream.getCurrentPath(), entryStream.getPosition());
263 }
264 batch.setMoreEntries(!replicationQueueInfo.isQueueRecovered() || moreData);
265 }
266
267 private void logMessage(WALEntryBatch batch) {
268 if (LOG.isTraceEnabled()) {
269 if (batch.isEmpty()) {
270 LOG.trace("Didn't read any new entries from WAL");
271 } else {
272 LOG.trace(String.format("Read %s WAL entries eligible for replication",
273 batch.getNbEntries()));
274 }
275 }
276 }
277
278 private boolean isShippable(WALEntryBatch batch) {
279 return !batch.isEmpty() || checkIfWALRolled(batch) || !batch.hasMoreEntries();
280 }
281
282 private boolean checkIfWALRolled(WALEntryBatch batch) {
283 return lastReadPath == null && batch.lastWalPath != null
284 || lastReadPath != null && !lastReadPath.equals(batch.lastWalPath);
285 }
286
287 private void resetStream(WALEntryStream stream) throws IOException {
288 lastReadPosition = stream.getPosition();
289 lastReadPath = stream.getCurrentPath();
290 stream.reset();
291 }
292
293
294
295
296
297
298
299
300
301
302
303 private boolean handleEofException(Exception e, WALEntryStream entryStream,
304 WALEntryBatch batch) throws InterruptedException {
305 boolean isRecoveredSource = manager.getOldSources().contains(source);
306 PriorityBlockingQueue<Path> queue = logQueue.getQueue(walGroupId);
307
308
309 if (e.getCause() instanceof EOFException && (isRecoveredSource || queue.size() > 1)
310 && conf.getBoolean("replication.source.eof.autorecovery", false)) {
311 Path path = queue.peek();
312 try {
313 if (!fs.exists(path)) {
314
315 path = entryStream.getArchivedLog(path);
316 }
317 if (fs.getFileStatus(path).getLen() == 0) {
318 LOG.warn("Forcing removal of 0 length log in queue: " + path);
319 lastReadPath = path;
320 logQueue.remove(walGroupId);
321 lastReadPosition = 0;
322
323
324
325 boolean hasMoreData = !queue.isEmpty();
326
327
328
329 updateBatchAndShippingQueue(entryStream, batch, hasMoreData, true);
330 return true;
331 }
332 } catch (IOException ioe) {
333 LOG.warn("Couldn't get file length information about log " + path, ioe);
334 }
335 }
336
337 return false;
338 }
339
340 public Path getCurrentPath() {
341 return logQueue.getQueue(walGroupId).peek();
342 }
343
344
345 private boolean checkQuota() {
346
347 if (totalBufferUsed.get() > totalBufferQuota) {
348 Threads.sleep(sleepForRetries);
349 return false;
350 }
351 return true;
352 }
353
354 private Entry filterEntry(Entry entry) {
355 Entry filtered = filter.filter(entry);
356 if (entry != null && filtered == null) {
357 metrics.incrLogEditsFiltered();
358 }
359 return filtered;
360 }
361
362
363
364
365
366
367
368 public WALEntryBatch take() throws InterruptedException {
369 return entryBatchQueue.take();
370 }
371
372 public WALEntryBatch poll(long timeout) throws InterruptedException {
373 return entryBatchQueue.poll(timeout, TimeUnit.MILLISECONDS);
374 }
375
376 private long getEntrySizeIncludeBulkLoad(Entry entry) {
377 WALEdit edit = entry.getEdit();
378 return getEntrySizeExcludeBulkLoad(entry) + sizeOfStoreFilesIncludeBulkLoad(edit);
379 }
380
381 public long getEntrySizeExcludeBulkLoad(Entry entry) {
382 WALEdit edit = entry.getEdit();
383 WALKey key = entry.getKey();
384 return edit.heapSize() + key.estimatedSerializedSizeOf();
385 }
386
387 private void updateBatchStats(WALEntryBatch batch, Entry entry,
388 long entryPosition, long entrySize) {
389 WALEdit edit = entry.getEdit();
390 if (edit != null && !edit.isEmpty()) {
391 batch.incrementHeapSize(entrySize);
392 Pair<Integer, Integer> nbRowsAndHFiles = countDistinctRowKeysAndHFiles(edit);
393 batch.incrementNbRowKeys(nbRowsAndHFiles.getFirst());
394 batch.incrementNbHFiles(nbRowsAndHFiles.getSecond());
395 }
396 batch.lastWalPosition = entryPosition;
397 }
398
399
400
401
402
403
404
405 private Pair<Integer, Integer> countDistinctRowKeysAndHFiles(WALEdit edit) {
406 List<Cell> cells = edit.getCells();
407 int distinctRowKeys = 1;
408 int totalHFileEntries = 0;
409 Cell lastCell = cells.get(0);
410
411 int totalCells = edit.size();
412 for (int i = 0; i < totalCells; i++) {
413
414 if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) {
415 try {
416 BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i));
417 List<StoreDescriptor> stores = bld.getStoresList();
418 int totalStores = stores.size();
419 for (int j = 0; j < totalStores; j++) {
420 totalHFileEntries += stores.get(j).getStoreFileList().size();
421 }
422 } catch (IOException e) {
423 LOG.error("Failed to deserialize bulk load entry from wal edit. "
424 + "Then its hfiles count will not be added into metric.");
425 }
426 }
427
428 if (!CellUtil.matchingRow(cells.get(i), lastCell)) {
429 distinctRowKeys++;
430 }
431 lastCell = cells.get(i);
432 }
433
434 Pair<Integer, Integer> result = new Pair<>(distinctRowKeys, totalHFileEntries);
435 return result;
436 }
437
438
439
440
441
442
443 private int sizeOfStoreFilesIncludeBulkLoad(WALEdit edit) {
444 List<Cell> cells = edit.getCells();
445 int totalStoreFilesSize = 0;
446
447 int totalCells = edit.size();
448 for (int i = 0; i < totalCells; i++) {
449 if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) {
450 try {
451 BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i));
452 List<StoreDescriptor> stores = bld.getStoresList();
453 int totalStores = stores.size();
454 for (int j = 0; j < totalStores; j++) {
455 totalStoreFilesSize += stores.get(j).getStoreFileSizeBytes();
456 }
457 } catch (IOException e) {
458 LOG.error("Failed to deserialize bulk load entry from wal edit. "
459 + "Size of HFiles part of cell will not be considered in replication "
460 + "request size calculation.",
461 e);
462 }
463 }
464 }
465 return totalStoreFilesSize;
466 }
467
468
469
470
471
472 private boolean acquireBufferQuota(long size) {
473 return totalBufferUsed.addAndGet(size) >= totalBufferQuota;
474 }
475
476
477
478
479 public boolean isReaderRunning() {
480 return isReaderRunning && !isInterrupted();
481 }
482
483
484
485
486 public void setReaderRunning(boolean readerRunning) {
487 this.isReaderRunning = readerRunning;
488 }
489
490 public long getLastReadPosition() {
491 return this.lastReadPosition;
492 }
493
494
495
496
497
498 final static class WALEntryBatch {
499 private List<Pair<Entry, Long>> walEntriesWithSize;
500
501 private Path lastWalPath;
502
503 private long lastWalPosition = 0;
504
505 private int nbRowKeys = 0;
506
507 private int nbHFiles = 0;
508
509 private long heapSize = 0;
510
511 private boolean moreEntries = true;
512
513
514
515
516 private WALEntryBatch(int maxNbEntries) {
517 this.walEntriesWithSize = new ArrayList<>(maxNbEntries);
518 }
519
520 public void addEntry(Entry entry, long entrySize) {
521 walEntriesWithSize.add(new Pair<>(entry, entrySize));
522 }
523
524
525
526
527 public List<Entry> getWalEntries() {
528 List<Entry> entries = new ArrayList<>(walEntriesWithSize.size());
529 for (Pair<Entry, Long> pair: walEntriesWithSize) {
530 entries.add(pair.getFirst());
531 }
532 return entries;
533 }
534
535
536
537
538 public List<Pair<Entry, Long>> getWalEntriesWithSize() {
539 return walEntriesWithSize;
540 }
541
542
543
544
545 public Path getLastWalPath() {
546 return lastWalPath;
547 }
548
549
550
551
552 public long getLastWalPosition() {
553 return lastWalPosition;
554 }
555
556 public int getNbEntries() {
557 return walEntriesWithSize.size();
558 }
559
560
561
562
563 public int getNbRowKeys() {
564 return nbRowKeys;
565 }
566
567
568
569
570 public int getNbHFiles() {
571 return nbHFiles;
572 }
573
574
575
576
577 public int getNbOperations() {
578 return getNbRowKeys() + getNbHFiles();
579 }
580
581
582
583
584 public long getHeapSize() {
585 return heapSize;
586 }
587
588 private void incrementNbRowKeys(int increment) {
589 nbRowKeys += increment;
590 }
591
592 private void incrementNbHFiles(int increment) {
593 nbHFiles += increment;
594 }
595
596 private void incrementHeapSize(long increment) {
597 heapSize += increment;
598 }
599
600 public boolean isEmpty() {
601 return walEntriesWithSize.isEmpty();
602 }
603
604
605
606
607
608
609
610 public void updatePosition(Path currentPath, long currentPosition) {
611 lastWalPath = currentPath;
612 lastWalPosition = currentPosition;
613 }
614
615 public boolean hasMoreEntries() {
616 return moreEntries;
617 }
618
619 public void setMoreEntries(boolean moreEntries) {
620 this.moreEntries = moreEntries;
621 }
622 }
623 }