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