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.FileNotFoundException;
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.Collection;
26 import java.util.Collections;
27 import java.util.List;
28 import java.util.concurrent.atomic.AtomicLong;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.hbase.Cell;
33 import org.apache.hadoop.hbase.CellUtil;
34 import org.apache.hadoop.hbase.HConstants;
35 import org.apache.hadoop.hbase.KeyValue;
36 import org.apache.hadoop.hbase.KeyValueUtil;
37 import org.apache.hadoop.hbase.classification.InterfaceAudience;
38 import org.apache.hadoop.hbase.client.Scan;
39 import org.apache.hadoop.hbase.io.TimeRange;
40 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
41 import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
42 import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
43
44
45
46
47
48 @InterfaceAudience.LimitedPrivate("Coprocessor")
49 public class StoreFileScanner implements KeyValueScanner {
50 private static final Log LOG = LogFactory.getLog(HStore.class);
51
52
53 private final StoreFile.Reader reader;
54 private final HFileScanner hfs;
55 private Cell cur = null;
56 private boolean closed = false;
57
58 private boolean realSeekDone;
59 private boolean delayedReseek;
60 private Cell delayedSeekKV;
61
62 private final boolean enforceMVCC;
63 private final boolean hasMVCCInfo;
64
65
66 private boolean stopSkippingKVsIfNextRow = false;
67
68 private static AtomicLong seekCount;
69
70 private final boolean canOptimizeForNonNullColumn;
71
72 private final long readPt;
73
74
75
76 private final long scannerOrder;
77
78
79
80
81
82
83
84
85
86
87
88 public StoreFileScanner(StoreFile.Reader reader, HFileScanner hfs, boolean useMVCC,
89 boolean hasMVCC, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) {
90 this.readPt = readPt;
91 this.reader = reader;
92 this.hfs = hfs;
93 this.enforceMVCC = useMVCC;
94 this.hasMVCCInfo = hasMVCC;
95 this.scannerOrder = scannerOrder;
96 this.canOptimizeForNonNullColumn = canOptimizeForNonNullColumn;
97 this.reader.incrementRefCount();
98 }
99
100 boolean isPrimaryReplica() {
101 return reader.isPrimaryReplicaReader();
102 }
103
104
105
106
107
108 public static List<StoreFileScanner> getScannersForStoreFiles(
109 Collection<StoreFile> files,
110 boolean cacheBlocks,
111 boolean usePread, long readPt) throws IOException {
112 return getScannersForStoreFiles(files, cacheBlocks,
113 usePread, false, false, readPt);
114 }
115
116
117
118
119 public static List<StoreFileScanner> getScannersForStoreFiles(
120 Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
121 boolean isCompaction, boolean useDropBehind, long readPt) throws IOException {
122 return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction,
123 useDropBehind, null, readPt);
124 }
125
126
127
128
129
130 public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
131 boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
132 ScanQueryMatcher matcher, long readPt, boolean isPrimaryReplica) throws IOException {
133 List<StoreFileScanner> scanners = new ArrayList<StoreFileScanner>(files.size());
134 List<StoreFile> sorted_files = new ArrayList<>(files);
135 Collections.sort(sorted_files, StoreFile.Comparators.SEQ_ID);
136 boolean succ = false;
137 try {
138 for (int i = 0; i < sorted_files.size(); i++) {
139 StoreFile.Reader r = sorted_files.get(i).createReader(canUseDrop);
140 r.setReplicaStoreFile(isPrimaryReplica);
141 StoreFileScanner scanner = r.getStoreFileScanner(cacheBlocks, usePread, isCompaction, readPt,
142 i, matcher != null ? !matcher.hasNullColumnInQuery() : false);
143 scanners.add(scanner);
144 }
145 succ = true;
146 } finally {
147 if (!succ) {
148 for (StoreFileScanner scanner : scanners) {
149 scanner.close();
150 }
151 }
152 }
153 return scanners;
154 }
155
156 public static List<StoreFileScanner> getScannersForStoreFiles(
157 Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
158 boolean isCompaction, boolean canUseDrop,
159 ScanQueryMatcher matcher, long readPt) throws IOException {
160 return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction, canUseDrop,
161 matcher, readPt, true);
162 }
163
164 @Override
165 public String toString() {
166 return "StoreFileScanner[" + hfs.toString() + ", cur=" + cur + "]";
167 }
168
169 @Override
170 public Cell peek() {
171 return cur;
172 }
173
174 @Override
175 public Cell next() throws IOException {
176 Cell retKey = cur;
177
178 try {
179
180 if (cur != null) {
181 hfs.next();
182 setCurrentCell(hfs.getKeyValue());
183 if (hasMVCCInfo || this.reader.isBulkLoaded()) {
184 skipKVsNewerThanReadpoint();
185 }
186 }
187 } catch (FileNotFoundException e) {
188 throw e;
189 } catch(IOException e) {
190 throw new IOException("Could not iterate " + this, e);
191 }
192 return retKey;
193 }
194
195 @Override
196 public boolean seek(Cell key) throws IOException {
197 if (seekCount != null) seekCount.incrementAndGet();
198
199 try {
200 try {
201 if(!seekAtOrAfter(hfs, key)) {
202 this.cur = null;
203 return false;
204 }
205
206 setCurrentCell(hfs.getKeyValue());
207
208 if (!hasMVCCInfo && this.reader.isBulkLoaded()) {
209 return skipKVsNewerThanReadpoint();
210 } else {
211 return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint();
212 }
213 } finally {
214 realSeekDone = true;
215 }
216 } catch (FileNotFoundException e) {
217 throw e;
218 } catch (IOException ioe) {
219 throw new IOException("Could not seek " + this + " to key " + key, ioe);
220 }
221 }
222
223 @Override
224 public boolean reseek(Cell key) throws IOException {
225 if (seekCount != null) seekCount.incrementAndGet();
226
227 try {
228 try {
229 if (!reseekAtOrAfter(hfs, key)) {
230 this.cur = null;
231 return false;
232 }
233 setCurrentCell(hfs.getKeyValue());
234
235 if (!hasMVCCInfo && this.reader.isBulkLoaded()) {
236 return skipKVsNewerThanReadpoint();
237 } else {
238 return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint();
239 }
240 } finally {
241 realSeekDone = true;
242 }
243 } catch (FileNotFoundException e) {
244 throw e;
245 } catch (IOException ioe) {
246 throw new IOException("Could not reseek " + this + " to key " + key,
247 ioe);
248 }
249 }
250
251 protected void setCurrentCell(Cell newVal) throws IOException {
252 this.cur = newVal;
253 if (this.cur != null && this.reader.isBulkLoaded()) {
254 CellUtil.setSequenceId(cur, this.reader.getSequenceID());
255 }
256 }
257
258 protected boolean skipKVsNewerThanReadpoint() throws IOException {
259
260
261 Cell startKV = cur;
262 while(enforceMVCC
263 && cur != null
264 && (cur.getMvccVersion() > readPt)) {
265 boolean hasNext = hfs.next();
266 setCurrentCell(hfs.getKeyValue());
267 if (hasNext && this.stopSkippingKVsIfNextRow
268 && getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(),
269 cur.getRowLength(), startKV.getRowArray(), startKV.getRowOffset(),
270 startKV.getRowLength()) > 0) {
271 return false;
272 }
273 }
274
275 if (cur == null) {
276 return false;
277 }
278
279 return true;
280 }
281
282 @Override
283 public void close() {
284 cur = null;
285 if (closed) return;
286 if (this.reader != null) {
287 this.reader.decrementRefCount();
288 }
289 closed = true;
290 this.hfs.close();
291 }
292
293
294
295
296
297
298
299
300 public static boolean seekAtOrAfter(HFileScanner s, Cell k)
301 throws IOException {
302 int result = s.seekTo(k);
303 if(result < 0) {
304 if (result == HConstants.INDEX_KEY_MAGIC) {
305
306 return true;
307 }
308
309 return s.seekTo();
310 } else if(result > 0) {
311
312
313 return s.next();
314 }
315
316 return true;
317 }
318
319 static boolean reseekAtOrAfter(HFileScanner s, Cell k)
320 throws IOException {
321
322 int result = s.reseekTo(k);
323 if (result <= 0) {
324 if (result == HConstants.INDEX_KEY_MAGIC) {
325
326 return true;
327 }
328
329
330
331 if (!s.isSeeked()) {
332 return s.seekTo();
333 }
334 return true;
335 }
336
337
338 return s.next();
339 }
340
341
342
343
344 @Override
345 public long getScannerOrder() {
346 return scannerOrder;
347 }
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363 @Override
364 public boolean requestSeek(Cell kv, boolean forward, boolean useBloom)
365 throws IOException {
366 if (kv.getFamilyLength() == 0) {
367 useBloom = false;
368 }
369
370 boolean haveToSeek = true;
371 if (useBloom) {
372
373 if (reader.getBloomFilterType() == BloomType.ROWCOL) {
374 haveToSeek = reader.passesGeneralBloomFilter(kv.getRowArray(),
375 kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(),
376 kv.getQualifierOffset(), kv.getQualifierLength());
377 } else if (canOptimizeForNonNullColumn
378 && ((CellUtil.isDeleteFamily(kv) || CellUtil.isDeleteFamilyVersion(kv)))) {
379
380
381 haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(), kv.getRowOffset(),
382 kv.getRowLength());
383 }
384 }
385
386 delayedReseek = forward;
387 delayedSeekKV = kv;
388
389 if (haveToSeek) {
390
391
392 realSeekDone = false;
393 long maxTimestampInFile = reader.getMaxTimestamp();
394 long seekTimestamp = kv.getTimestamp();
395 if (seekTimestamp > maxTimestampInFile) {
396
397
398
399
400
401
402 setCurrentCell(KeyValueUtil.createFirstOnRowColTS(kv, maxTimestampInFile));
403 } else {
404
405
406
407
408 enforceSeek();
409 }
410 return cur != null;
411 }
412
413
414
415
416
417
418
419
420 setCurrentCell(KeyValueUtil.createLastOnRowCol(kv));
421
422 realSeekDone = true;
423 return true;
424 }
425
426 Reader getReader() {
427 return reader;
428 }
429
430 KeyValue.KVComparator getComparator() {
431 return reader.getComparator();
432 }
433
434 @Override
435 public boolean realSeekDone() {
436 return realSeekDone;
437 }
438
439 @Override
440 public void enforceSeek() throws IOException {
441 if (realSeekDone)
442 return;
443
444 if (delayedReseek) {
445 reseek(delayedSeekKV);
446 } else {
447 seek(delayedSeekKV);
448 }
449 }
450
451 @Override
452 public boolean isFileScanner() {
453 return true;
454 }
455
456
457
458 static final long getSeekCount() {
459 return seekCount.get();
460 }
461 static final void instrument() {
462 seekCount = new AtomicLong();
463 }
464
465 @Override
466 public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
467
468 byte[] cf = store.getFamily().getName();
469 TimeRange timeRange = scan.getColumnFamilyTimeRange().get(cf);
470 if (timeRange == null) {
471 timeRange = scan.getTimeRange();
472 }
473 return reader.passesTimerangeFilter(timeRange, oldestUnexpiredTS) && reader
474 .passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan, scan.getFamilyMap().get(cf));
475 }
476
477 @Override
478 @SuppressWarnings("deprecation")
479 public boolean seekToPreviousRow(Cell originalKey) throws IOException {
480 try {
481 try {
482 boolean keepSeeking = false;
483 Cell key = originalKey;
484 do {
485 KeyValue seekKey = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
486 key.getRowLength());
487 if (seekCount != null) seekCount.incrementAndGet();
488 if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(),
489 seekKey.getKeyLength())) {
490 this.cur = null;
491 return false;
492 }
493 KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue()
494 .getRowArray(), hfs.getKeyValue().getRowOffset(), hfs.getKeyValue().getRowLength());
495
496 if (seekCount != null) seekCount.incrementAndGet();
497 if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
498 close();
499 return false;
500 }
501
502 setCurrentCell(hfs.getKeyValue());
503 this.stopSkippingKVsIfNextRow = true;
504 boolean resultOfSkipKVs;
505 try {
506 resultOfSkipKVs = skipKVsNewerThanReadpoint();
507 } finally {
508 this.stopSkippingKVsIfNextRow = false;
509 }
510 if (!resultOfSkipKVs
511 || getComparator().compareRows(cur, firstKeyOfPreviousRow) > 0) {
512 keepSeeking = true;
513 key = firstKeyOfPreviousRow;
514 continue;
515 } else {
516 keepSeeking = false;
517 }
518 } while (keepSeeking);
519 return true;
520 } finally {
521 realSeekDone = true;
522 }
523 } catch (FileNotFoundException e) {
524 throw e;
525 } catch (IOException ioe) {
526 throw new IOException("Could not seekToPreviousRow " + this + " to key "
527 + originalKey, ioe);
528 }
529 }
530
531 @Override
532 public boolean seekToLastRow() throws IOException {
533 byte[] lastRow = reader.getLastRowKey();
534 if (lastRow == null) {
535 return false;
536 }
537 KeyValue seekKey = KeyValueUtil.createFirstOnRow(lastRow);
538 if (seek(seekKey)) {
539 return true;
540 } else {
541 return seekToPreviousRow(seekKey);
542 }
543 }
544
545 @Override
546 public boolean backwardSeek(Cell key) throws IOException {
547 seek(key);
548 if (cur == null
549 || getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(),
550 cur.getRowLength(), key.getRowArray(), key.getRowOffset(),
551 key.getRowLength()) > 0) {
552 return seekToPreviousRow(key);
553 }
554 return true;
555 }
556
557 @Override
558 public Cell getNextIndexedKey() {
559 return hfs.getNextIndexedKey();
560 }
561 }