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.regionserver;
21
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.util.ArrayList;
25 import java.util.List;
26 import java.util.NavigableSet;
27 import java.util.concurrent.CountDownLatch;
28 import java.util.concurrent.locks.ReentrantLock;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.classification.InterfaceAudience;
34 import org.apache.hadoop.hbase.Cell;
35 import org.apache.hadoop.hbase.CellUtil;
36 import org.apache.hadoop.hbase.DoNotRetryIOException;
37 import org.apache.hadoop.hbase.HConstants;
38 import org.apache.hadoop.hbase.KeyValue;
39 import org.apache.hadoop.hbase.KeyValue.KVComparator;
40 import org.apache.hadoop.hbase.KeyValueUtil;
41 import org.apache.hadoop.hbase.client.IsolationLevel;
42 import org.apache.hadoop.hbase.client.Scan;
43 import org.apache.hadoop.hbase.executor.ExecutorService;
44 import org.apache.hadoop.hbase.filter.Filter;
45 import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
46 import org.apache.hadoop.hbase.regionserver.handler.ParallelSeekHandler;
47 import org.apache.hadoop.hbase.util.Bytes;
48 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
49
50
51
52
53
54 @InterfaceAudience.Private
55 public class StoreScanner extends NonReversedNonLazyKeyValueScanner
56 implements KeyValueScanner, InternalScanner, ChangedReadersObserver {
57 static final Log LOG = LogFactory.getLog(StoreScanner.class);
58 protected Store store;
59 protected ScanQueryMatcher matcher;
60 protected KeyValueHeap heap;
61 protected boolean cacheBlocks;
62
63 protected int countPerRow = 0;
64 protected int storeLimit = -1;
65 protected int storeOffset = 0;
66
67
68
69 protected boolean closing = false;
70 protected final boolean isGet;
71 protected final boolean explicitColumnQuery;
72 protected final boolean useRowColBloom;
73
74
75
76 protected boolean isParallelSeekEnabled = false;
77 protected ExecutorService executor;
78 protected final Scan scan;
79 protected final NavigableSet<byte[]> columns;
80 protected final long oldestUnexpiredTS;
81 protected final long now;
82 protected final int minVersions;
83 protected final long maxRowSize;
84
85
86
87
88
89 private long kvsScanned = 0;
90 private Cell prevCell = null;
91
92
93 static final boolean LAZY_SEEK_ENABLED_BY_DEFAULT = true;
94 public static final String STORESCANNER_PARALLEL_SEEK_ENABLE =
95 "hbase.storescanner.parallel.seek.enable";
96
97
98 protected static boolean lazySeekEnabledGlobally =
99 LAZY_SEEK_ENABLED_BY_DEFAULT;
100
101
102 protected Cell lastTop = null;
103
104
105 private boolean scanUsePread = false;
106 protected ReentrantLock lock = new ReentrantLock();
107
108 private final long readPt;
109
110
111 enum StoreScannerCompactionRace {
112 BEFORE_SEEK,
113 AFTER_SEEK,
114 COMPACT_COMPLETE
115 }
116
117
118 protected StoreScanner(Store store, boolean cacheBlocks, Scan scan,
119 final NavigableSet<byte[]> columns, long ttl, int minVersions, long readPt) {
120 this.readPt = readPt;
121 this.store = store;
122 this.cacheBlocks = cacheBlocks;
123 isGet = scan.isGetScan();
124 int numCol = columns == null ? 0 : columns.size();
125 explicitColumnQuery = numCol > 0;
126 this.scan = scan;
127 this.columns = columns;
128 this.now = EnvironmentEdgeManager.currentTime();
129 this.oldestUnexpiredTS = now - ttl;
130 this.minVersions = minVersions;
131
132 if (store != null && ((HStore)store).getHRegion() != null
133 && ((HStore)store).getHRegion().getBaseConf() != null) {
134 Configuration conf = ((HStore) store).getHRegion().getBaseConf();
135 this.maxRowSize =
136 conf.getLong(HConstants.TABLE_MAX_ROWSIZE_KEY, HConstants.TABLE_MAX_ROWSIZE_DEFAULT);
137 this.scanUsePread = conf.getBoolean("hbase.storescanner.use.pread", scan.isSmall());
138 } else {
139 this.maxRowSize = HConstants.TABLE_MAX_ROWSIZE_DEFAULT;
140 this.scanUsePread = scan.isSmall();
141 }
142
143
144
145
146
147 useRowColBloom = numCol > 1 || (!isGet && numCol == 1);
148
149
150
151
152 if (store != null && ((HStore)store).getHRegion() != null
153 && store.getStorefilesCount() > 1) {
154 RegionServerServices rsService = ((HStore)store).getHRegion().getRegionServerServices();
155 if (rsService == null || !rsService.getConfiguration().getBoolean(
156 STORESCANNER_PARALLEL_SEEK_ENABLE, false)) return;
157 isParallelSeekEnabled = true;
158 executor = rsService.getExecutorService();
159 }
160 }
161
162
163
164
165
166
167
168
169
170
171 public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, final NavigableSet<byte[]> columns,
172 long readPt)
173 throws IOException {
174 this(store, scan.getCacheBlocks(), scan, columns, scanInfo.getTtl(),
175 scanInfo.getMinVersions(), readPt);
176 if (columns != null && scan.isRaw()) {
177 throw new DoNotRetryIOException(
178 "Cannot specify any column for a raw scan");
179 }
180 matcher = new ScanQueryMatcher(scan, scanInfo, columns,
181 ScanType.USER_SCAN, Long.MAX_VALUE, HConstants.LATEST_TIMESTAMP,
182 oldestUnexpiredTS, now, store.getCoprocessorHost());
183
184 this.store.addChangedReaderObserver(this);
185
186
187 List<KeyValueScanner> scanners = getScannersNoCompaction();
188
189
190
191
192
193 seekScanners(scanners, matcher.getStartKey(), explicitColumnQuery
194 && lazySeekEnabledGlobally, isParallelSeekEnabled);
195
196
197 this.storeLimit = scan.getMaxResultsPerColumnFamily();
198
199
200 this.storeOffset = scan.getRowOffsetPerColumnFamily();
201
202
203 resetKVHeap(scanners, store.getComparator());
204 }
205
206
207
208
209
210
211
212
213
214
215
216 public StoreScanner(Store store, ScanInfo scanInfo, Scan scan,
217 List<? extends KeyValueScanner> scanners, ScanType scanType,
218 long smallestReadPoint, long earliestPutTs) throws IOException {
219 this(store, scanInfo, scan, scanners, scanType, smallestReadPoint, earliestPutTs, null, null);
220 }
221
222
223
224
225
226
227
228
229
230
231
232
233 public StoreScanner(Store store, ScanInfo scanInfo, Scan scan,
234 List<? extends KeyValueScanner> scanners, long smallestReadPoint, long earliestPutTs,
235 byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
236 this(store, scanInfo, scan, scanners, ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint,
237 earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
238 }
239
240 private StoreScanner(Store store, ScanInfo scanInfo, Scan scan,
241 List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
242 long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
243 this(store, false, scan, null, scanInfo.getTtl(), scanInfo.getMinVersions(),
244 ((HStore)store).getHRegion().getReadpoint(IsolationLevel.READ_COMMITTED));
245 if (dropDeletesFromRow == null) {
246 matcher = new ScanQueryMatcher(scan, scanInfo, null, scanType, smallestReadPoint,
247 earliestPutTs, oldestUnexpiredTS, now, store.getCoprocessorHost());
248 } else {
249 matcher = new ScanQueryMatcher(scan, scanInfo, null, smallestReadPoint, earliestPutTs,
250 oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow, store.getCoprocessorHost());
251 }
252
253
254 scanners = selectScannersFrom(scanners);
255
256
257 seekScanners(scanners, matcher.getStartKey(), false, isParallelSeekEnabled);
258
259
260 resetKVHeap(scanners, store.getComparator());
261 }
262
263
264 StoreScanner(final Scan scan, ScanInfo scanInfo,
265 ScanType scanType, final NavigableSet<byte[]> columns,
266 final List<KeyValueScanner> scanners) throws IOException {
267 this(scan, scanInfo, scanType, columns, scanners,
268 HConstants.LATEST_TIMESTAMP,
269
270 0);
271 }
272
273
274 StoreScanner(final Scan scan, ScanInfo scanInfo,
275 ScanType scanType, final NavigableSet<byte[]> columns,
276 final List<KeyValueScanner> scanners, long earliestPutTs)
277 throws IOException {
278 this(scan, scanInfo, scanType, columns, scanners, earliestPutTs,
279
280 0);
281 }
282
283 private StoreScanner(final Scan scan, ScanInfo scanInfo,
284 ScanType scanType, final NavigableSet<byte[]> columns,
285 final List<KeyValueScanner> scanners, long earliestPutTs, long readPt)
286 throws IOException {
287 this(null, scan.getCacheBlocks(), scan, columns, scanInfo.getTtl(),
288 scanInfo.getMinVersions(), readPt);
289 this.matcher = new ScanQueryMatcher(scan, scanInfo, columns, scanType,
290 Long.MAX_VALUE, earliestPutTs, oldestUnexpiredTS, now, null);
291
292
293 if (this.store != null) {
294 this.store.addChangedReaderObserver(this);
295 }
296
297 seekScanners(scanners, matcher.getStartKey(), false, isParallelSeekEnabled);
298 resetKVHeap(scanners, scanInfo.getComparator());
299 }
300
301
302
303
304
305 protected List<KeyValueScanner> getScannersNoCompaction() throws IOException {
306 final boolean isCompaction = false;
307 boolean usePread = isGet || scanUsePread;
308 return selectScannersFrom(store.getScanners(cacheBlocks, isGet, usePread,
309 isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt));
310 }
311
312
313
314
315
316
317
318
319
320 protected void seekScanners(List<? extends KeyValueScanner> scanners,
321 Cell seekKey, boolean isLazy, boolean isParallelSeek)
322 throws IOException {
323
324
325
326
327 if (isLazy) {
328 for (KeyValueScanner scanner : scanners) {
329 scanner.requestSeek(seekKey, false, true);
330 }
331 } else {
332 if (!isParallelSeek) {
333 long totalScannersSoughtBytes = 0;
334 for (KeyValueScanner scanner : scanners) {
335 if (totalScannersSoughtBytes >= maxRowSize) {
336 throw new RowTooBigException("Max row size allowed: " + maxRowSize
337 + ", but row is bigger than that");
338 }
339 scanner.seek(seekKey);
340 Cell c = scanner.peek();
341 if (c != null) {
342 totalScannersSoughtBytes += CellUtil.estimatedSerializedSizeOf(c);
343 }
344 }
345 } else {
346 parallelSeek(scanners, seekKey);
347 }
348 }
349 }
350
351 protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
352 KVComparator comparator) throws IOException {
353
354 heap = new KeyValueHeap(scanners, comparator);
355 }
356
357
358
359
360
361 protected List<KeyValueScanner> selectScannersFrom(
362 final List<? extends KeyValueScanner> allScanners) {
363 boolean memOnly;
364 boolean filesOnly;
365 if (scan instanceof InternalScan) {
366 InternalScan iscan = (InternalScan)scan;
367 memOnly = iscan.isCheckOnlyMemStore();
368 filesOnly = iscan.isCheckOnlyStoreFiles();
369 } else {
370 memOnly = false;
371 filesOnly = false;
372 }
373
374 List<KeyValueScanner> scanners =
375 new ArrayList<KeyValueScanner>(allScanners.size());
376
377
378
379 long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS :
380 Long.MIN_VALUE;
381
382
383 for (KeyValueScanner kvs : allScanners) {
384 boolean isFile = kvs.isFileScanner();
385 if ((!isFile && filesOnly) || (isFile && memOnly)) {
386 continue;
387 }
388
389 if (kvs.shouldUseScanner(scan, columns, expiredTimestampCutoff)) {
390 scanners.add(kvs);
391 }
392 }
393 return scanners;
394 }
395
396 @Override
397 public Cell peek() {
398 lock.lock();
399 try {
400 if (this.heap == null) {
401 return this.lastTop;
402 }
403 return this.heap.peek();
404 } finally {
405 lock.unlock();
406 }
407 }
408
409 @Override
410 public KeyValue next() {
411
412 throw new RuntimeException("Never call StoreScanner.next()");
413 }
414
415 @Override
416 public void close() {
417 lock.lock();
418 try {
419 if (this.closing) return;
420 this.closing = true;
421
422 if (this.store != null)
423 this.store.deleteChangedReaderObserver(this);
424 if (this.heap != null)
425 this.heap.close();
426 this.heap = null;
427 this.lastTop = null;
428 } finally {
429 lock.unlock();
430 }
431 }
432
433 @Override
434 public boolean seek(Cell key) throws IOException {
435 lock.lock();
436 try {
437
438 checkReseek();
439 return this.heap.seek(key);
440 } finally {
441 lock.unlock();
442 }
443 }
444
445
446
447
448
449
450
451 @Override
452 public boolean next(List<Cell> outResult, int limit) throws IOException {
453 lock.lock();
454 try {
455 if (checkReseek()) {
456 return true;
457 }
458
459
460
461 if (this.heap == null) {
462 close();
463 return false;
464 }
465
466 Cell peeked = this.heap.peek();
467 if (peeked == null) {
468 close();
469 return false;
470 }
471
472
473
474 byte[] row = peeked.getRowArray();
475 int offset = peeked.getRowOffset();
476 short length = peeked.getRowLength();
477 if (limit < 0 || matcher.row == null || !Bytes.equals(row, offset, length, matcher.row,
478 matcher.rowOffset, matcher.rowLength)) {
479 this.countPerRow = 0;
480 matcher.setRow(row, offset, length);
481 }
482
483 Cell cell;
484
485
486 KeyValue.KVComparator comparator =
487 store != null ? store.getComparator() : null;
488
489 int count = 0;
490 long totalBytesRead = 0;
491
492 LOOP: while((cell = this.heap.peek()) != null) {
493 if (prevCell != cell) ++kvsScanned;
494 checkScanOrder(prevCell, cell, comparator);
495 prevCell = cell;
496
497 ScanQueryMatcher.MatchCode qcode = matcher.match(cell);
498 qcode = optimize(qcode, cell);
499 switch(qcode) {
500 case INCLUDE:
501 case INCLUDE_AND_SEEK_NEXT_ROW:
502 case INCLUDE_AND_SEEK_NEXT_COL:
503
504 Filter f = matcher.getFilter();
505 if (f != null) {
506
507 cell = f.transformCell(cell);
508 }
509
510 this.countPerRow++;
511 if (storeLimit > -1 &&
512 this.countPerRow > (storeLimit + storeOffset)) {
513
514 if (!matcher.moreRowsMayExistAfter(cell)) {
515 return false;
516 }
517 seekToNextRow(cell);
518 break LOOP;
519 }
520
521
522
523 if (this.countPerRow > storeOffset) {
524 outResult.add(cell);
525 count++;
526 totalBytesRead += CellUtil.estimatedSerializedSizeOf(cell);
527 if (totalBytesRead > maxRowSize) {
528 throw new RowTooBigException("Max row size allowed: " + maxRowSize
529 + ", but the row is bigger than that.");
530 }
531 }
532
533 if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
534 if (!matcher.moreRowsMayExistAfter(cell)) {
535 return false;
536 }
537 seekToNextRow(cell);
538 } else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
539 seekAsDirection(matcher.getKeyForNextColumn(cell));
540 } else {
541 this.heap.next();
542 }
543
544 if (limit > 0 && (count == limit)) {
545 break LOOP;
546 }
547 continue;
548
549 case DONE:
550 return true;
551
552 case DONE_SCAN:
553 close();
554 return false;
555
556 case SEEK_NEXT_ROW:
557
558
559 if (!matcher.moreRowsMayExistAfter(cell)) {
560 return false;
561 }
562
563 seekToNextRow(cell);
564 break;
565
566 case SEEK_NEXT_COL:
567 seekAsDirection(matcher.getKeyForNextColumn(cell));
568 break;
569
570 case SKIP:
571 this.heap.next();
572 break;
573
574 case SEEK_NEXT_USING_HINT:
575
576 Cell nextKV = matcher.getNextKeyHint(cell);
577 if (nextKV != null) {
578 seekAsDirection(nextKV);
579 } else {
580 heap.next();
581 }
582 break;
583
584 default:
585 throw new RuntimeException("UNEXPECTED");
586 }
587 }
588
589 if (count > 0) {
590 return true;
591 }
592
593
594 close();
595 return false;
596 } finally {
597 lock.unlock();
598 }
599 }
600
601
602
603
604
605 private ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
606 Cell nextIndexedKey = getNextIndexedKey();
607 if (nextIndexedKey == null || nextIndexedKey == HConstants.NO_NEXT_INDEXED_KEY ||
608 store == null) {
609 return qcode;
610 }
611 switch(qcode) {
612 case INCLUDE_AND_SEEK_NEXT_COL:
613 case SEEK_NEXT_COL:
614 {
615 if (matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0) {
616 return qcode == MatchCode.SEEK_NEXT_COL ? MatchCode.SKIP : MatchCode.INCLUDE;
617 }
618 break;
619 }
620 case INCLUDE_AND_SEEK_NEXT_ROW:
621 case SEEK_NEXT_ROW:
622 {
623 if (matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0) {
624 return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
625 }
626 break;
627 }
628 default:
629 break;
630 }
631 return qcode;
632 }
633
634 @Override
635 public boolean next(List<Cell> outResult) throws IOException {
636 return next(outResult, -1);
637 }
638
639
640 @Override
641 public void updateReaders() throws IOException {
642 lock.lock();
643 try {
644 if (this.closing) return;
645
646
647
648
649
650
651 if (this.heap == null) return;
652
653
654 this.lastTop = this.peek();
655
656
657
658
659 this.heap.close();
660 this.heap = null;
661
662
663 } finally {
664 lock.unlock();
665 }
666 }
667
668
669
670
671
672
673 protected boolean checkReseek() throws IOException {
674 if (this.heap == null && this.lastTop != null) {
675 resetScannerStack(this.lastTop);
676 if (this.heap.peek() == null
677 || store.getComparator().compareRows(this.lastTop, this.heap.peek()) != 0) {
678 LOG.debug("Storescanner.peek() is changed where before = "
679 + this.lastTop.toString() + ",and after = " + this.heap.peek());
680 this.lastTop = null;
681 return true;
682 }
683 this.lastTop = null;
684 }
685
686 return false;
687 }
688
689 protected void resetScannerStack(Cell lastTopKey) throws IOException {
690 if (heap != null) {
691 throw new RuntimeException("StoreScanner.reseek run on an existing heap!");
692 }
693
694
695
696
697 List<KeyValueScanner> scanners = getScannersNoCompaction();
698
699
700 seekScanners(scanners, lastTopKey, false, isParallelSeekEnabled);
701
702
703 resetKVHeap(scanners, store.getComparator());
704
705
706
707
708 Cell kv = heap.peek();
709 if (kv == null) {
710 kv = lastTopKey;
711 }
712 byte[] row = kv.getRowArray();
713 int offset = kv.getRowOffset();
714 short length = kv.getRowLength();
715 if ((matcher.row == null) || !Bytes.equals(row, offset, length, matcher.row,
716 matcher.rowOffset, matcher.rowLength)) {
717 this.countPerRow = 0;
718 matcher.reset();
719 matcher.setRow(row, offset, length);
720 }
721 }
722
723
724
725
726
727
728
729
730 protected void checkScanOrder(Cell prevKV, Cell kv,
731 KeyValue.KVComparator comparator) throws IOException {
732
733 assert prevKV == null || comparator == null
734 || comparator.compare(prevKV, kv) <= 0 : "Key " + prevKV
735 + " followed by a " + "smaller key " + kv + " in cf " + store;
736 }
737
738 protected boolean seekToNextRow(Cell kv) throws IOException {
739 return reseek(KeyValueUtil.createLastOnRow(kv));
740 }
741
742
743
744
745
746
747
748 protected boolean seekAsDirection(Cell kv)
749 throws IOException {
750 return reseek(kv);
751 }
752
753 @Override
754 public boolean reseek(Cell kv) throws IOException {
755 lock.lock();
756 try {
757
758
759
760 checkReseek();
761 if (explicitColumnQuery && lazySeekEnabledGlobally) {
762 return heap.requestSeek(kv, true, useRowColBloom);
763 }
764 return heap.reseek(kv);
765 } finally {
766 lock.unlock();
767 }
768 }
769
770 @Override
771 public long getSequenceID() {
772 return 0;
773 }
774
775
776
777
778
779
780
781 private void parallelSeek(final List<? extends KeyValueScanner>
782 scanners, final Cell kv) throws IOException {
783 if (scanners.isEmpty()) return;
784 int storeFileScannerCount = scanners.size();
785 CountDownLatch latch = new CountDownLatch(storeFileScannerCount);
786 List<ParallelSeekHandler> handlers =
787 new ArrayList<ParallelSeekHandler>(storeFileScannerCount);
788 for (KeyValueScanner scanner : scanners) {
789 if (scanner instanceof StoreFileScanner) {
790 ParallelSeekHandler seekHandler = new ParallelSeekHandler(scanner, kv,
791 this.readPt, latch);
792 executor.submit(seekHandler);
793 handlers.add(seekHandler);
794 } else {
795 scanner.seek(kv);
796 latch.countDown();
797 }
798 }
799
800 try {
801 latch.await();
802 } catch (InterruptedException ie) {
803 throw (InterruptedIOException)new InterruptedIOException().initCause(ie);
804 }
805
806 for (ParallelSeekHandler handler : handlers) {
807 if (handler.getErr() != null) {
808 throw new IOException(handler.getErr());
809 }
810 }
811 }
812
813
814
815
816
817 List<KeyValueScanner> getAllScannersForTesting() {
818 List<KeyValueScanner> allScanners = new ArrayList<KeyValueScanner>();
819 KeyValueScanner current = heap.getCurrentForTesting();
820 if (current != null)
821 allScanners.add(current);
822 for (KeyValueScanner scanner : heap.getHeap())
823 allScanners.add(scanner);
824 return allScanners;
825 }
826
827 static void enableLazySeekGlobally(boolean enable) {
828 lazySeekEnabledGlobally = enable;
829 }
830
831
832
833
834 public long getEstimatedNumberOfKvsScanned() {
835 return this.kvsScanned;
836 }
837
838 @Override
839 public Cell getNextIndexedKey() {
840 return this.heap.getNextIndexedKey();
841 }
842 }
843