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