@InterfaceAudience.LimitedPrivate(value="Phoenix") @InterfaceStability.Evolving public class StoreFileScanner extends Object implements KeyValueScanner
Modifier and Type | Field and Description |
---|---|
private boolean |
canOptimizeForNonNullColumn |
private boolean |
closed |
private Cell |
cur |
private boolean |
delayedReseek |
private Cell |
delayedSeekKV |
private boolean |
enforceMVCC |
private boolean |
hasMVCCInfo |
private HFileScanner |
hfs |
private StoreFileReader |
reader |
private long |
readPt |
private boolean |
realSeekDone |
private long |
scannerOrder |
private static LongAdder |
seekCount |
private boolean |
stopSkippingKVsIfNextRow |
NO_NEXT_INDEXED_KEY
Constructor and Description |
---|
StoreFileScanner(StoreFileReader reader,
HFileScanner hfs,
boolean useMVCC,
boolean hasMVCC,
long readPt,
long scannerOrder,
boolean canOptimizeForNonNullColumn)
Implements a
KeyValueScanner on top of the specified HFileScanner |
Modifier and Type | Method and Description |
---|---|
boolean |
backwardSeek(Cell key)
Seek the scanner at or before the row of specified Cell, it firstly tries to seek the scanner
at or after the specified Cell, return if peek KeyValue of scanner has the same row with
specified Cell, otherwise seek the scanner at the first Cell of the row which is the previous
row of specified KeyValue
|
void |
close()
Close the KeyValue scanner.
|
void |
enforceSeek()
Does the real seek operation in case it was skipped by seekToRowCol(KeyValue, boolean) (TODO:
Whats this?).
|
(package private) CellComparator |
getComparator() |
org.apache.hadoop.fs.Path |
getFilePath() |
Cell |
getNextIndexedKey() |
(package private) StoreFileReader |
getReader() |
long |
getScannerOrder()
Get the order of this KeyValueScanner.
|
static List<StoreFileScanner> |
getScannersForCompaction(Collection<HStoreFile> files,
boolean canUseDropBehind,
long readPt)
Get scanners for compaction.
|
static List<StoreFileScanner> |
getScannersForStoreFiles(Collection<HStoreFile> files,
boolean cacheBlocks,
boolean usePread,
boolean isCompaction,
boolean useDropBehind,
long readPt)
Return an array of scanners corresponding to the given set of store files.
|
static List<StoreFileScanner> |
getScannersForStoreFiles(Collection<HStoreFile> files,
boolean cacheBlocks,
boolean usePread,
boolean isCompaction,
boolean canUseDrop,
ScanQueryMatcher matcher,
long readPt)
Return an array of scanners corresponding to the given set of store files, And set the
ScanQueryMatcher for each store file scanner for further optimization
|
(package private) static long |
getSeekCount() |
(package private) static void |
instrument() |
boolean |
isFileScanner()
Returns true if this is a file scanner.
|
Cell |
next()
Return the next Cell in this scanner, iterating the scanner
|
Cell |
peek()
Look at the next Cell in this scanner, but do not iterate scanner.
|
boolean |
realSeekDone()
We optimize our store scanners by checking the most recent store file first, so we sometimes
pretend we have done a seek but delay it until the store scanner bubbles up to the top of the
key-value heap.
|
boolean |
requestSeek(Cell kv,
boolean forward,
boolean useBloom)
Pretend we have done a seek but don't do it yet, if possible.
|
boolean |
reseek(Cell key)
Reseek the scanner at or after the specified KeyValue.
|
(package private) static boolean |
reseekAtOrAfter(HFileScanner s,
Cell k) |
boolean |
seek(Cell key)
Seek the scanner at or after the specified KeyValue.
|
static boolean |
seekAtOrAfter(HFileScanner s,
Cell k)
nn * @return false if not found or if k is after the end.
|
boolean |
seekToLastRow()
Seek the scanner at the first KeyValue of last row
|
boolean |
seekToPreviousRow(Cell originalKey)
Seek the scanner at the first Cell of the row which is the previous row of specified key
|
protected void |
setCurrentCell(Cell newVal) |
void |
shipped()
Called after a batch of rows scanned and set to be returned to client.
|
boolean |
shouldUseScanner(Scan scan,
HStore store,
long oldestUnexpiredTS)
Allows to filter out scanners (both StoreFile and memstore) that we don't want to use based on
criteria such as Bloom filters and timestamp ranges.
|
protected boolean |
skipKVsNewerThanReadpoint() |
String |
toString() |
private final StoreFileReader reader
private final HFileScanner hfs
private boolean closed
private boolean realSeekDone
private boolean delayedReseek
private Cell delayedSeekKV
private final boolean enforceMVCC
private final boolean hasMVCCInfo
private boolean stopSkippingKVsIfNextRow
private final boolean canOptimizeForNonNullColumn
private final long readPt
private final long scannerOrder
public StoreFileScanner(StoreFileReader reader, HFileScanner hfs, boolean useMVCC, boolean hasMVCC, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
KeyValueScanner
on top of the specified HFileScanner
useMVCC
- If true, scanner will filter out updates with MVCC larger
than readPt
.readPt
- MVCC value to use to filter out the updates newer than this
scanner.hasMVCC
- Set to true if underlying store file reader has MVCC info.scannerOrder
- Order of the scanner relative to other scanners. See
KeyValueScanner.getScannerOrder()
.canOptimizeForNonNullColumn
- true
if we can make sure there is no null column,
otherwise false
. This is a hint for optimization.public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files, boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean useDropBehind, long readPt) throws IOException
IOException
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files, boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop, ScanQueryMatcher matcher, long readPt) throws IOException
IOException
public static List<StoreFileScanner> getScannersForCompaction(Collection<HStoreFile> files, boolean canUseDropBehind, long readPt) throws IOException
IOException
public Cell peek()
KeyValueScanner
peek
in interface KeyValueScanner
public Cell next() throws IOException
KeyValueScanner
next
in interface KeyValueScanner
IOException
public boolean seek(Cell key) throws IOException
KeyValueScanner
seek
in interface KeyValueScanner
key
- seek valueIOException
public boolean reseek(Cell key) throws IOException
KeyValueScanner
reseek
in interface KeyValueScanner
key
- seek value (should be non-null)IOException
protected void setCurrentCell(Cell newVal) throws IOException
IOException
protected boolean skipKVsNewerThanReadpoint() throws IOException
IOException
public void close()
KeyValueScanner
close
in interface Closeable
close
in interface AutoCloseable
close
in interface KeyValueScanner
public static boolean seekAtOrAfter(HFileScanner s, Cell k) throws IOException
IOException
static boolean reseekAtOrAfter(HFileScanner s, Cell k) throws IOException
IOException
public long getScannerOrder()
KeyValueScanner
getScannerOrder
in interface KeyValueScanner
KeyValueScanner.getScannerOrder()
public boolean requestSeek(Cell kv, boolean forward, boolean useBloom) throws IOException
enforceSeek()
.
Note that this function does guarantee that the current KV of this scanner will be advanced to at least the given KV. Because of this, it does have to do a real seek in cases when the seek timestamp is older than the highest timestamp of the file, e.g. when we are trying to seek to the next row/column and use OLDEST_TIMESTAMP in the seek key.
requestSeek
in interface KeyValueScanner
forward
- do a forward-only "reseek" instead of a random-access seekuseBloom
- whether to enable multi-column Bloom filter optimizationIOException
StoreFileReader getReader()
CellComparator getComparator()
public boolean realSeekDone()
KeyValueScanner
realSeekDone
in interface KeyValueScanner
public void enforceSeek() throws IOException
KeyValueScanner
KeyValueScanner.realSeekDone()
first.enforceSeek
in interface KeyValueScanner
IOException
public boolean isFileScanner()
KeyValueScanner
isFileScanner
in interface KeyValueScanner
public org.apache.hadoop.fs.Path getFilePath()
getFilePath
in interface KeyValueScanner
KeyValueScanner.isFileScanner()
static final long getSeekCount()
static final void instrument()
public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS)
KeyValueScanner
shouldUseScanner
in interface KeyValueScanner
scan
- the scan that we are selecting scanners forstore
- the store we are performing the scan on.oldestUnexpiredTS
- the oldest timestamp we are interested in for this query, based on TTLpublic boolean seekToPreviousRow(Cell originalKey) throws IOException
KeyValueScanner
seekToPreviousRow
in interface KeyValueScanner
originalKey
- seek valueIOException
public boolean seekToLastRow() throws IOException
KeyValueScanner
seekToLastRow
in interface KeyValueScanner
IOException
public boolean backwardSeek(Cell key) throws IOException
KeyValueScanner
backwardSeek
in interface KeyValueScanner
key
- seek KeyValueIOException
public Cell getNextIndexedKey()
getNextIndexedKey
in interface KeyValueScanner
public void shipped() throws IOException
Shipper
shipped
in interface Shipper
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.