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