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