Interface KeyValueScanner
- All Superinterfaces:
AutoCloseable
,Closeable
,Shipper
- All Known Implementing Classes:
CollectionBackedScanner
,KeyValueHeap
,MobStoreScanner
,NonLazyKeyValueScanner
,NonReversedNonLazyKeyValueScanner
,ReversedKeyValueHeap
,ReversedMobStoreScanner
,ReversedStoreScanner
,SegmentScanner
,SnapshotSegmentScanner
,StoreFileScanner
,StoreScanner
Scanner that returns the next KeyValue.
-
Field Summary
Modifier and TypeFieldDescriptionstatic final ExtendedCell
The byte array represents for NO_NEXT_INDEXED_KEY; The actual value is irrelevant because this is always compared by reference. -
Method Summary
Modifier and TypeMethodDescriptionboolean
backwardSeek
(ExtendedCell 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 KeyValuevoid
close()
Close the KeyValue scanner.void
Does the real seek operation in case it was skipped by seekToRowCol(KeyValue, boolean) (TODO: Whats this?).org.apache.hadoop.fs.Path
default long
Get the order of this KeyValueScanner.boolean
Returns true if this is a file scanner.next()
Return the next Cell in this scanner, iterating the scannerpeek()
Look at the next Cell in this scanner, but do not iterate scanner.boolean
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.void
recordBlockSize
(IntConsumer blockSizeConsumer) Record the size of the current block in bytes, passing as an argument to the blockSizeConsumer.boolean
requestSeek
(ExtendedCell kv, boolean forward, boolean useBloom) Similar toseek(org.apache.hadoop.hbase.ExtendedCell)
(orreseek(org.apache.hadoop.hbase.ExtendedCell)
if forward is true) but only does a seek operation after checking that it is really necessary for the row/column combination specified by the kv parameter.boolean
reseek
(ExtendedCell key) Reseek the scanner at or after the specified KeyValue.boolean
seek
(ExtendedCell key) Seek the scanner at or after the specified KeyValue.boolean
Seek the scanner at the first KeyValue of last rowboolean
Seek the scanner at the first Cell of the row which is the previous row of specified keyboolean
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.
-
Field Details
-
NO_NEXT_INDEXED_KEY
The byte array represents for NO_NEXT_INDEXED_KEY; The actual value is irrelevant because this is always compared by reference.
-
-
Method Details
-
peek
Look at the next Cell in this scanner, but do not iterate scanner. NOTICE: The returned cell has not been passed into ScanQueryMatcher. So it may not be what the user need.- Returns:
- the next Cell
-
next
Return the next Cell in this scanner, iterating the scanner- Returns:
- the next Cell
- Throws:
IOException
-
seek
Seek the scanner at or after the specified KeyValue.- Parameters:
key
- seek value- Returns:
- true if scanner has values left, false if end of scanner
- Throws:
IOException
-
reseek
Reseek the scanner at or after the specified KeyValue. This method is guaranteed to seek at or after the required key only if the key comes after the current position of the scanner. Should not be used to seek to a key which may come before the current position.- Parameters:
key
- seek value (should be non-null)- Returns:
- true if scanner has values left, false if end of scanner
- Throws:
IOException
-
getScannerOrder
Get the order of this KeyValueScanner. This is only relevant for StoreFileScanners. This is required for comparing multiple files to find out which one has the latest data. StoreFileScanners are ordered from 0 (oldest) to newest in increasing order. -
close
void close()Close the KeyValue scanner.- Specified by:
close
in interfaceAutoCloseable
- Specified by:
close
in interfaceCloseable
-
shouldUseScanner
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.- Parameters:
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 TTL- Returns:
- true if the scanner should be included in the query
-
requestSeek
Similar toseek(org.apache.hadoop.hbase.ExtendedCell)
(orreseek(org.apache.hadoop.hbase.ExtendedCell)
if forward is true) but only does a seek operation after checking that it is really necessary for the row/column combination specified by the kv parameter. This function was added to avoid unnecessary disk seeks by checking row-column Bloom filters before a seek on multi-column get/scan queries, and to optimize by looking up more recent files first.- Parameters:
forward
- do a forward-only "reseek" instead of a random-access seekuseBloom
- whether to enable multi-column Bloom filter optimization- Throws:
IOException
-
realSeekDone
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. This method is then used to ensure the top store file scanner has done a seek operation. -
enforceSeek
Does the real seek operation in case it was skipped by seekToRowCol(KeyValue, boolean) (TODO: Whats this?). Note that this function should be never called on scanners that always do real seek operations (i.e. most of the scanners). The easiest way to achieve this is to callrealSeekDone()
first.- Throws:
IOException
-
isFileScanner
boolean isFileScanner()Returns true if this is a file scanner. Otherwise a memory scanner is assumed. -
recordBlockSize
Record the size of the current block in bytes, passing as an argument to the blockSizeConsumer. Implementations should ensure that blockSizeConsumer is only called once per block.- Parameters:
blockSizeConsumer
- to be called with block size in bytes, once per block.
-
getFilePath
org.apache.hadoop.fs.Path getFilePath()- Returns:
- the file path if this is a file scanner, otherwise null.
- See Also:
-
backwardSeek
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- Parameters:
key
- seek KeyValue- Returns:
- true if the scanner is at the valid KeyValue, false if such KeyValue does not exist
- Throws:
IOException
-
seekToPreviousRow
Seek the scanner at the first Cell of the row which is the previous row of specified key- Parameters:
key
- seek value- Returns:
- true if the scanner at the first valid Cell of previous row, false if not existing such Cell
- Throws:
IOException
-
seekToLastRow
Seek the scanner at the first KeyValue of last row- Returns:
- true if scanner has values left, false if the underlying data is empty
- Throws:
IOException
-
getNextIndexedKey
- Returns:
- the next key in the index, usually the first key of next block OR a key that falls between last key of current block and first key of next block.. see HFileWriterImpl#getMidpoint, or null if not known.
-