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