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