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.hbase.Cell;
33 import org.apache.hadoop.hbase.CellUtil;
34 import org.apache.hadoop.hbase.DoNotRetryIOException;
35 import org.apache.hadoop.hbase.HConstants;
36 import org.apache.hadoop.hbase.KeyValue;
37 import org.apache.hadoop.hbase.KeyValue.KVComparator;
38 import org.apache.hadoop.hbase.KeyValueUtil;
39 import org.apache.hadoop.hbase.classification.InterfaceAudience;
40 import org.apache.hadoop.hbase.client.IsolationLevel;
41 import org.apache.hadoop.hbase.client.Scan;
42 import org.apache.hadoop.hbase.executor.ExecutorService;
43 import org.apache.hadoop.hbase.filter.Filter;
44 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
45 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
46 import org.apache.hadoop.hbase.regionserver.handler.ParallelSeekHandler;
47 import org.apache.hadoop.hbase.regionserver.querymatcher.CompactionScanQueryMatcher;
48 import org.apache.hadoop.hbase.regionserver.querymatcher.LegacyScanQueryMatcher;
49 import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
50 import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
51 import org.apache.hadoop.hbase.util.Bytes;
52 import org.apache.hadoop.hbase.util.CollectionUtils;
53 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
54
55
56
57
58
59 @InterfaceAudience.Private
60 public class StoreScanner extends NonReversedNonLazyKeyValueScanner
61 implements KeyValueScanner, InternalScanner, ChangedReadersObserver {
62 private static final Log LOG = LogFactory.getLog(StoreScanner.class);
63
64 protected final Store store;
65 protected ScanQueryMatcher matcher;
66 protected KeyValueHeap heap;
67 protected boolean cacheBlocks;
68
69 protected long countPerRow = 0;
70 protected int storeLimit = -1;
71 protected int storeOffset = 0;
72
73
74
75 protected boolean closing = false;
76 protected final boolean get;
77 protected final boolean explicitColumnQuery;
78 protected final boolean useRowColBloom;
79
80
81
82 protected boolean parallelSeekEnabled = false;
83 protected ExecutorService executor;
84 protected final Scan scan;
85 protected final NavigableSet<byte[]> columns;
86 protected final long oldestUnexpiredTS;
87 protected final long now;
88 protected final int minVersions;
89 protected final long maxRowSize;
90 protected final long cellsPerHeartbeatCheck;
91
92
93
94
95
96 private final List<KeyValueScanner> scannersForDelayedClose = new ArrayList<>();
97
98
99
100
101 private long kvsScanned = 0;
102 private Cell prevCell = null;
103
104
105 static final boolean LAZY_SEEK_ENABLED_BY_DEFAULT = true;
106 public static final String STORESCANNER_PARALLEL_SEEK_ENABLE =
107 "hbase.storescanner.parallel.seek.enable";
108
109
110 protected static boolean lazySeekEnabledGlobally =
111 LAZY_SEEK_ENABLED_BY_DEFAULT;
112
113
114
115
116
117
118 public static final String HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK =
119 "hbase.cells.scanned.per.heartbeat.check";
120
121
122
123
124 public static final long DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK = 10000;
125
126
127 protected Cell lastTop = null;
128
129
130 private boolean scanUsePread = false;
131
132 private volatile boolean flushed = false;
133
134
135 private final List<KeyValueScanner> flushedstoreFileScanners =
136 new ArrayList<KeyValueScanner>(1);
137
138 private final List<KeyValueScanner> memStoreScannersAfterFlush = new ArrayList<>(1);
139
140 private final List<KeyValueScanner> currentScanners = new ArrayList<KeyValueScanner>();
141
142 private ReentrantLock flushLock = new ReentrantLock();
143
144 private final long readPt;
145 private boolean topChanged = false;
146
147
148 enum StoreScannerCompactionRace {
149 BEFORE_SEEK,
150 AFTER_SEEK,
151 COMPACT_COMPLETE
152 }
153
154
155 protected StoreScanner(Store store, Scan scan, final ScanInfo scanInfo,
156 final NavigableSet<byte[]> columns, long readPt, boolean cacheBlocks) {
157 this.readPt = readPt;
158 this.store = store;
159 this.cacheBlocks = cacheBlocks;
160 get = scan.isGetScan();
161 int numCol = columns == null ? 0 : columns.size();
162 explicitColumnQuery = numCol > 0;
163 this.scan = scan;
164 this.columns = columns;
165 this.now = EnvironmentEdgeManager.currentTime();
166 this.oldestUnexpiredTS = scan.isRaw() ? 0L : now - scanInfo.getTtl();
167 this.minVersions = scanInfo.getMinVersions();
168
169
170
171
172
173 this.useRowColBloom = (numCol > 1 || (!get && numCol == 1))
174 && (store == null || store.getFamily().getBloomFilterType() == BloomType.ROWCOL);
175
176 this.maxRowSize = scanInfo.getTableMaxRowSize();
177 this.scanUsePread = scan.isSmall()? true: scanInfo.isUsePread();
178 this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck();
179
180 if (this.store != null && this.store.getStorefilesCount() > 1) {
181 RegionServerServices rsService = ((HStore)store).getHRegion().getRegionServerServices();
182 if (rsService != null && scanInfo.isParallelSeekEnabled()) {
183 this.parallelSeekEnabled = true;
184 this.executor = rsService.getExecutorService();
185 }
186 }
187 }
188
189 protected void addCurrentScanners(List<? extends KeyValueScanner> scanners) {
190 this.currentScanners.addAll(scanners);
191 }
192
193
194
195
196
197
198
199
200
201
202 public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, final NavigableSet<byte[]> columns,
203 long readPt)
204 throws IOException {
205 this(store, scan, scanInfo, columns, readPt, scan.getCacheBlocks());
206 if (columns != null && scan.isRaw()) {
207 throw new DoNotRetryIOException("Cannot specify any column for a raw scan");
208 }
209 matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
210 store.getCoprocessorHost());
211
212 this.store.addChangedReaderObserver(this);
213
214 try {
215
216 List<KeyValueScanner> scanners = getScannersNoCompaction();
217
218
219
220
221
222 seekScanners(scanners, matcher.getStartKey(), explicitColumnQuery && lazySeekEnabledGlobally,
223 parallelSeekEnabled);
224
225
226 this.storeLimit = scan.getMaxResultsPerColumnFamily();
227
228
229 this.storeOffset = scan.getRowOffsetPerColumnFamily();
230 addCurrentScanners(scanners);
231
232 resetKVHeap(scanners, store.getComparator());
233 } catch (IOException e) {
234
235
236 this.store.deleteChangedReaderObserver(this);
237 throw e;
238 }
239 }
240
241
242
243
244
245
246
247
248
249
250
251 public StoreScanner(Store store, ScanInfo scanInfo, Scan scan,
252 List<? extends KeyValueScanner> scanners, ScanType scanType,
253 long smallestReadPoint, long earliestPutTs) throws IOException {
254 this(store, scanInfo, scan, scanners, scanType, smallestReadPoint, earliestPutTs, null, null);
255 }
256
257
258
259
260
261
262
263
264
265
266
267
268 public StoreScanner(Store store, ScanInfo scanInfo, Scan scan,
269 List<? extends KeyValueScanner> scanners, long smallestReadPoint, long earliestPutTs,
270 byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
271 this(store, scanInfo, scan, scanners, ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint,
272 earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
273 }
274
275 private StoreScanner(Store store, ScanInfo scanInfo, Scan scan,
276 List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
277 long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
278 this(store, scan, scanInfo, null,
279 ((HStore) store).getHRegion().getReadpoint(IsolationLevel.READ_COMMITTED), false);
280 if (scan.hasFilter() || (scan.getStartRow() != null && scan.getStartRow().length > 0)
281 || (scan.getStopRow() != null && scan.getStopRow().length > 0)
282 || !scan.getTimeRange().isAllTime()) {
283
284
285 matcher = LegacyScanQueryMatcher.create(scan, scanInfo, null, scanType, smallestReadPoint,
286 earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow,
287 store.getCoprocessorHost());
288 } else {
289 matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, smallestReadPoint,
290 earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow,
291 store.getCoprocessorHost());
292 }
293
294
295 scanners = selectScannersFrom(scanners);
296
297
298 seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
299 addCurrentScanners(scanners);
300
301 resetKVHeap(scanners, store.getComparator());
302 }
303
304 StoreScanner(final Scan scan, ScanInfo scanInfo,
305 ScanType scanType, final NavigableSet<byte[]> columns,
306 final List<KeyValueScanner> scanners) throws IOException {
307 this(scan, scanInfo, scanType, columns, scanners,
308 HConstants.LATEST_TIMESTAMP,
309
310 0);
311 }
312
313 StoreScanner(final Scan scan, ScanInfo scanInfo,
314 ScanType scanType, final NavigableSet<byte[]> columns,
315 final List<KeyValueScanner> scanners, long earliestPutTs)
316 throws IOException {
317 this(scan, scanInfo, scanType, columns, scanners, earliestPutTs,
318
319 0);
320 }
321
322 public StoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType,
323 final NavigableSet<byte[]> columns, final List<KeyValueScanner> scanners, long earliestPutTs,
324 long readPt) throws IOException {
325 this(null, scan, scanInfo, columns, readPt, scan.getCacheBlocks());
326 if (scanType == ScanType.USER_SCAN) {
327 this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
328 null);
329 } else {
330 if (scan.hasFilter() || (scan.getStartRow() != null && scan.getStartRow().length > 0)
331 || (scan.getStopRow() != null && scan.getStopRow().length > 0)
332 || !scan.getTimeRange().isAllTime() || columns != null) {
333
334
335 matcher = LegacyScanQueryMatcher.create(scan, scanInfo, columns, scanType, Long.MAX_VALUE,
336 earliestPutTs, oldestUnexpiredTS, now, null, null, store.getCoprocessorHost());
337 } else {
338 this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,
339 earliestPutTs, oldestUnexpiredTS, now, null, null, null);
340 }
341 }
342
343
344 seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
345 addCurrentScanners(scanners);
346 resetKVHeap(scanners, scanInfo.getComparator());
347 }
348
349
350
351
352
353 protected List<KeyValueScanner> getScannersNoCompaction() throws IOException {
354 final boolean isCompaction = false;
355 boolean usePread = get || scanUsePread;
356 return selectScannersFrom(store.getScanners(cacheBlocks, get, usePread,
357 isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt));
358 }
359
360
361
362
363
364
365
366
367
368 protected void seekScanners(List<? extends KeyValueScanner> scanners,
369 Cell seekKey, boolean isLazy, boolean isParallelSeek)
370 throws IOException {
371
372
373
374
375 if (isLazy) {
376 for (KeyValueScanner scanner : scanners) {
377 scanner.requestSeek(seekKey, false, true);
378 }
379 } else {
380 if (!isParallelSeek) {
381 long totalScannersSoughtBytes = 0;
382 for (KeyValueScanner scanner : scanners) {
383 if (matcher.isUserScan() && totalScannersSoughtBytes >= maxRowSize) {
384 throw new RowTooBigException("Max row size allowed: " + maxRowSize
385 + ", but row is bigger than that");
386 }
387 scanner.seek(seekKey);
388 Cell c = scanner.peek();
389 if (c != null) {
390 totalScannersSoughtBytes += CellUtil.estimatedSerializedSizeOf(c);
391 }
392 }
393 } else {
394 parallelSeek(scanners, seekKey);
395 }
396 }
397 }
398
399 protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
400 KVComparator comparator) throws IOException {
401
402 heap = new KeyValueHeap(scanners, comparator);
403 }
404
405
406
407
408
409 protected List<KeyValueScanner> selectScannersFrom(
410 final List<? extends KeyValueScanner> allScanners) {
411 boolean memOnly;
412 boolean filesOnly;
413 if (scan instanceof InternalScan) {
414 InternalScan iscan = (InternalScan)scan;
415 memOnly = iscan.isCheckOnlyMemStore();
416 filesOnly = iscan.isCheckOnlyStoreFiles();
417 } else {
418 memOnly = false;
419 filesOnly = false;
420 }
421
422 List<KeyValueScanner> scanners =
423 new ArrayList<KeyValueScanner>(allScanners.size());
424
425
426
427 long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS: Long.MIN_VALUE;
428
429
430 for (KeyValueScanner kvs : allScanners) {
431 boolean isFile = kvs.isFileScanner();
432 if ((!isFile && filesOnly) || (isFile && memOnly)) {
433 continue;
434 }
435
436 if (kvs.shouldUseScanner(scan, store, expiredTimestampCutoff)) {
437 scanners.add(kvs);
438 } else {
439 kvs.close();
440 }
441 }
442 return scanners;
443 }
444
445 @Override
446 public Cell peek() {
447 if (this.heap == null) {
448 return this.lastTop;
449 }
450 return this.heap.peek();
451 }
452
453 @Override
454 public KeyValue next() {
455
456 throw new RuntimeException("Never call StoreScanner.next()");
457 }
458
459 @Override
460 public void close() {
461 if (this.closing) {
462 return;
463 }
464
465
466 if (this.store != null) {
467 this.store.deleteChangedReaderObserver(this);
468 }
469
470
471
472 flushLock.lock();
473 try {
474 this.closing = true;
475 clearAndClose(scannersForDelayedClose);
476 clearAndClose(memStoreScannersAfterFlush);
477
478
479
480 clearAndClose(flushedstoreFileScanners);
481 } finally {
482 flushLock.unlock();
483 }
484 if (this.heap != null)
485 this.heap.close();
486 this.heap = null;
487 this.lastTop = null;
488 }
489
490 @Override
491 public boolean seek(Cell key) throws IOException {
492 boolean flushed = checkFlushed();
493
494 checkReseek(flushed);
495 return this.heap.seek(key);
496 }
497
498 @Override
499 public boolean next(List<Cell> outResult) throws IOException {
500 return next(outResult, NoLimitScannerContext.getInstance());
501 }
502
503
504
505
506
507
508
509 @Override
510 public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws IOException {
511 if (scannerContext == null) {
512 throw new IllegalArgumentException("Scanner context cannot be null");
513 }
514 boolean flushed = checkFlushed();
515 if (checkReseek(flushed)) {
516 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
517 }
518
519
520
521 if (this.heap == null) {
522 close();
523 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
524 }
525
526 Cell cell = this.heap.peek();
527 if (cell == null) {
528 close();
529 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
530 }
531
532
533
534
535
536
537
538 if (!scannerContext.hasAnyLimit(LimitScope.BETWEEN_CELLS) || matcher.currentRow() == null) {
539 this.countPerRow = 0;
540 matcher.setToNewRow(cell);
541 }
542
543
544 if (!scannerContext.getKeepProgress()) scannerContext.clearProgress();
545
546
547 KeyValue.KVComparator comparator =
548 store != null ? store.getComparator() : null;
549
550 int count = 0;
551 long totalBytesRead = 0;
552
553 LOOP: do {
554
555 if ((kvsScanned % cellsPerHeartbeatCheck == 0)) {
556 scannerContext.updateTimeProgress();
557 if (scannerContext.checkTimeLimit(LimitScope.BETWEEN_CELLS)) {
558 return scannerContext.setScannerState(NextState.TIME_LIMIT_REACHED).hasMoreValues();
559 }
560 }
561
562 if (prevCell != cell) ++kvsScanned;
563 checkScanOrder(prevCell, cell, comparator);
564 prevCell = cell;
565 scannerContext.setLastPeekedCell(cell);
566 topChanged = false;
567 ScanQueryMatcher.MatchCode qcode = matcher.match(cell);
568 switch (qcode) {
569 case INCLUDE:
570 case INCLUDE_AND_SEEK_NEXT_ROW:
571 case INCLUDE_AND_SEEK_NEXT_COL:
572
573 Filter f = matcher.getFilter();
574 if (f != null) {
575 cell = f.transformCell(cell);
576 }
577
578 this.countPerRow++;
579 if (storeLimit > -1 &&
580 this.countPerRow > (storeLimit + storeOffset)) {
581
582 if (!matcher.moreRowsMayExistAfter(cell)) {
583 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
584 }
585
586
587
588 matcher.clearCurrentRow();
589 seekToNextRow(cell);
590 break LOOP;
591 }
592
593
594
595 if (this.countPerRow > storeOffset) {
596 outResult.add(cell);
597
598
599 count++;
600 totalBytesRead += CellUtil.estimatedSerializedSizeOf(cell);
601
602
603 scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell));
604 scannerContext.incrementBatchProgress(1);
605
606 if (matcher.isUserScan() && totalBytesRead > maxRowSize) {
607 String message = "Max row size allowed: " + maxRowSize
608 + ", but the row is bigger than that, the row info: " + CellUtil
609 .toString(cell, false) + ", already have process row cells = " + outResult.size()
610 + ", it belong to region = " + store.getRegionInfo().getRegionNameAsString();
611 LOG.warn(message);
612 throw new RowTooBigException(message);
613 }
614 }
615
616 if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
617 if (!matcher.moreRowsMayExistAfter(cell)) {
618 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
619 }
620
621
622
623 matcher.clearCurrentRow();
624 seekOrSkipToNextRow(cell);
625 } else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
626 seekOrSkipToNextColumn(cell);
627 } else {
628 this.heap.next();
629 }
630
631 if (scannerContext.checkBatchLimit(LimitScope.BETWEEN_CELLS)) {
632 break LOOP;
633 }
634 if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_CELLS)) {
635 break LOOP;
636 }
637 continue;
638
639 case DONE:
640
641 if (this.scan.isGetScan()) {
642
643 close();
644 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
645 }
646
647
648
649 matcher.clearCurrentRow();
650 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
651
652 case DONE_SCAN:
653 close();
654 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
655
656 case SEEK_NEXT_ROW:
657
658
659 if (!matcher.moreRowsMayExistAfter(cell)) {
660 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
661 }
662
663
664
665 matcher.clearCurrentRow();
666 seekOrSkipToNextRow(cell);
667 NextState stateAfterSeekNextRow = needToReturn(outResult);
668 if (stateAfterSeekNextRow != null) {
669 return scannerContext.setScannerState(stateAfterSeekNextRow).hasMoreValues();
670 }
671 break;
672
673 case SEEK_NEXT_COL:
674 seekOrSkipToNextColumn(cell);
675 NextState stateAfterSeekNextColumn = needToReturn(outResult);
676 if (stateAfterSeekNextColumn != null) {
677 return scannerContext.setScannerState(stateAfterSeekNextColumn).hasMoreValues();
678 }
679 break;
680
681 case SKIP:
682 this.heap.next();
683 break;
684
685 case SEEK_NEXT_USING_HINT:
686 Cell nextKV = matcher.getNextKeyHint(cell);
687 if (nextKV != null && comparator.compare(nextKV, cell) > 0) {
688 seekAsDirection(nextKV);
689 NextState stateAfterSeekByHint = needToReturn(outResult);
690 if (stateAfterSeekByHint != null) {
691 return scannerContext.setScannerState(stateAfterSeekByHint).hasMoreValues();
692 }
693 } else {
694 heap.next();
695 }
696 break;
697
698 default:
699 throw new RuntimeException("UNEXPECTED");
700 }
701 } while((cell = this.heap.peek()) != null);
702
703 if (count > 0) {
704 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
705 }
706
707
708 close();
709 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
710 }
711
712
713
714
715
716
717
718
719
720
721
722
723 private NextState needToReturn(List<Cell> outResult) {
724 if (!outResult.isEmpty() && topChanged) {
725 return heap.peek() == null ? NextState.NO_MORE_VALUES : NextState.MORE_VALUES;
726 }
727 return null;
728 }
729
730 private void seekOrSkipToNextRow(Cell cell) throws IOException {
731
732
733 if (!get) {
734 if (trySkipToNextRow(cell)) {
735 return;
736 }
737 }
738 seekToNextRow(cell);
739 }
740
741 private void seekOrSkipToNextColumn(Cell cell) throws IOException {
742 if (!trySkipToNextColumn(cell)) {
743 seekAsDirection(matcher.getKeyForNextColumn(cell));
744 }
745 }
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796 protected boolean trySkipToNextRow(Cell cell) throws IOException {
797 Cell nextCell = null;
798
799
800 Cell previousIndexedKey = null;
801 do {
802 Cell nextIndexedKey = getNextIndexedKey();
803 if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&
804 (nextIndexedKey == previousIndexedKey ||
805 matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0)) {
806 this.heap.next();
807 ++kvsScanned;
808 previousIndexedKey = nextIndexedKey;
809 } else {
810 return false;
811 }
812 } while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRow(cell, nextCell));
813 return true;
814 }
815
816
817
818
819
820
821 protected boolean trySkipToNextColumn(Cell cell) throws IOException {
822 Cell nextCell = null;
823
824
825 Cell previousIndexedKey = null;
826 do {
827 Cell nextIndexedKey = getNextIndexedKey();
828 if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&
829 (nextIndexedKey == previousIndexedKey ||
830 matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0)) {
831 this.heap.next();
832 ++kvsScanned;
833 previousIndexedKey = nextIndexedKey;
834 } else {
835 return false;
836 }
837 } while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRowColumn(cell, nextCell));
838
839
840
841 if (useRowColBloom && nextCell != null && matcher.compareKeyForNextColumn(nextCell, cell) < 0) {
842 return false;
843 }
844 return true;
845 }
846
847 @Override
848 public long getReadPoint() {
849 return readPt;
850 }
851
852 private static void clearAndClose(List<KeyValueScanner> scanners) {
853 for (KeyValueScanner s : scanners) {
854 s.close();
855 }
856 scanners.clear();
857 }
858
859
860 @Override
861 public void updateReaders(List<StoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException {
862 if (CollectionUtils.isEmpty(sfs) && CollectionUtils.isEmpty(memStoreScanners)) {
863 return;
864 }
865 flushLock.lock();
866 try {
867 if (this.closing) {
868
869
870 if (!CollectionUtils.isEmpty(memStoreScanners)) {
871 clearAndClose(new ArrayList<>(memStoreScanners));
872 }
873 return;
874 }
875 flushed = true;
876 final boolean isCompaction = false;
877 boolean usePread = get || scanUsePread;
878
879
880
881
882
883
884 List<KeyValueScanner> scanners = store.getScanners(sfs, cacheBlocks, get, usePread,
885 isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, false);
886 flushedstoreFileScanners.addAll(scanners);
887 if (!CollectionUtils.isEmpty(memStoreScanners)) {
888 clearAndClose(memStoreScannersAfterFlush);
889 memStoreScannersAfterFlush.addAll(memStoreScanners);
890 }
891 } finally {
892 flushLock.unlock();
893 }
894 }
895
896
897 protected void nullifyCurrentHeap() throws IOException {
898 if (this.closing) return;
899
900
901
902
903
904
905 if (this.heap == null) return;
906
907
908 this.lastTop = this.heap.peek();
909
910
911
912
913 this.heap.close();
914 this.heap = null;
915
916
917 }
918
919
920
921
922
923
924
925 protected boolean checkReseek(boolean flushed) throws IOException {
926 if (flushed && this.lastTop != null) {
927 resetScannerStack(this.lastTop);
928 if (this.heap.peek() == null
929 || store.getComparator().compareRows(this.lastTop, this.heap.peek()) != 0) {
930 LOG.info("Storescanner.peek() is changed where before = "
931 + this.lastTop.toString() + ",and after = " + this.heap.peek());
932 this.lastTop = null;
933 topChanged = true;
934 return true;
935 }
936 this.lastTop = null;
937 }
938
939 topChanged = false;
940 return false;
941 }
942
943 protected void resetScannerStack(Cell lastTopKey) throws IOException {
944
945
946
947
948
949 final boolean isCompaction = false;
950 boolean usePread = get || scanUsePread;
951 List<KeyValueScanner> scanners = null;
952 flushLock.lock();
953 try {
954 List<KeyValueScanner> allScanners =
955 new ArrayList<>(flushedstoreFileScanners.size() + memStoreScannersAfterFlush.size());
956 allScanners.addAll(flushedstoreFileScanners);
957 allScanners.addAll(memStoreScannersAfterFlush);
958 scanners = selectScannersFrom(allScanners);
959
960 flushedstoreFileScanners.clear();
961 memStoreScannersAfterFlush.clear();
962 } finally {
963 flushLock.unlock();
964 }
965
966
967 seekScanners(scanners, lastTopKey, false, parallelSeekEnabled);
968
969 for (int i = 0; i < currentScanners.size(); i++) {
970 if (!currentScanners.get(i).isFileScanner()) {
971 scannersForDelayedClose.add(currentScanners.remove(i));
972 break;
973 }
974 }
975
976 addCurrentScanners(scanners);
977
978 resetKVHeap(this.currentScanners, store.getComparator());
979
980
981
982 Cell kv = heap.peek();
983 if (kv == null) {
984 kv = lastTopKey;
985 }
986 byte[] row = kv.getRowArray();
987 int offset = kv.getRowOffset();
988 short length = kv.getRowLength();
989 Cell currentRow = matcher.currentRow();
990
991 if ((currentRow == null) || !Bytes.equals(row, offset, length, currentRow.getRowArray(),
992 currentRow.getRowOffset(), currentRow.getRowLength())) {
993 this.countPerRow = 0;
994 matcher.setToNewRow(kv);
995 }
996 }
997
998
999
1000
1001
1002
1003
1004
1005 protected void checkScanOrder(Cell prevKV, Cell kv,
1006 KeyValue.KVComparator comparator) throws IOException {
1007
1008 assert prevKV == null || comparator == null
1009 || comparator.compare(prevKV, kv) <= 0 : "Key " + prevKV
1010 + " followed by a " + "smaller key " + kv + " in cf " + store;
1011 }
1012
1013 protected boolean seekToNextRow(Cell kv) throws IOException {
1014 return reseek(KeyValueUtil.createLastOnRow(kv));
1015 }
1016
1017
1018
1019
1020
1021
1022
1023 protected boolean seekAsDirection(Cell kv)
1024 throws IOException {
1025 return reseek(kv);
1026 }
1027
1028 @Override
1029 public boolean reseek(Cell kv) throws IOException {
1030 boolean flushed = checkFlushed();
1031
1032
1033
1034 checkReseek(flushed);
1035 if (explicitColumnQuery && lazySeekEnabledGlobally) {
1036 return heap.requestSeek(kv, true, useRowColBloom);
1037 }
1038 return heap.reseek(kv);
1039 }
1040
1041 protected boolean checkFlushed() {
1042
1043
1044
1045
1046
1047 if (flushed) {
1048
1049
1050 if(!this.closing) {
1051 this.lastTop = this.peek();
1052 } else {
1053 return false;
1054 }
1055
1056 flushed = false;
1057 return true;
1058 }
1059 return false;
1060 }
1061
1062
1063
1064
1065 @Override
1066 public long getScannerOrder() {
1067 return 0;
1068 }
1069
1070
1071
1072
1073
1074
1075
1076 private void parallelSeek(final List<? extends KeyValueScanner>
1077 scanners, final Cell kv) throws IOException {
1078 if (scanners.isEmpty()) return;
1079 int storeFileScannerCount = scanners.size();
1080 CountDownLatch latch = new CountDownLatch(storeFileScannerCount);
1081 List<ParallelSeekHandler> handlers =
1082 new ArrayList<ParallelSeekHandler>(storeFileScannerCount);
1083 for (KeyValueScanner scanner : scanners) {
1084 if (scanner instanceof StoreFileScanner) {
1085 ParallelSeekHandler seekHandler = new ParallelSeekHandler(scanner, kv,
1086 this.readPt, latch);
1087 executor.submit(seekHandler);
1088 handlers.add(seekHandler);
1089 } else {
1090 scanner.seek(kv);
1091 latch.countDown();
1092 }
1093 }
1094
1095 try {
1096 latch.await();
1097 } catch (InterruptedException ie) {
1098 throw (InterruptedIOException)new InterruptedIOException().initCause(ie);
1099 }
1100
1101 for (ParallelSeekHandler handler : handlers) {
1102 if (handler.getErr() != null) {
1103 throw new IOException(handler.getErr());
1104 }
1105 }
1106 }
1107
1108
1109
1110
1111
1112 List<KeyValueScanner> getAllScannersForTesting() {
1113 List<KeyValueScanner> allScanners = new ArrayList<KeyValueScanner>();
1114 KeyValueScanner current = heap.getCurrentForTesting();
1115 if (current != null)
1116 allScanners.add(current);
1117 for (KeyValueScanner scanner : heap.getHeap())
1118 allScanners.add(scanner);
1119 return allScanners;
1120 }
1121
1122 static void enableLazySeekGlobally(boolean enable) {
1123 lazySeekEnabledGlobally = enable;
1124 }
1125
1126
1127
1128
1129 public long getEstimatedNumberOfKvsScanned() {
1130 return this.kvsScanned;
1131 }
1132
1133 @Override
1134 public Cell getNextIndexedKey() {
1135 return this.heap.getNextIndexedKey();
1136 }
1137 }
1138