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