Class StoreScanner
java.lang.Object
org.apache.hadoop.hbase.regionserver.NonLazyKeyValueScanner
org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner
org.apache.hadoop.hbase.regionserver.StoreScanner
- All Implemented Interfaces:
Closeable
,AutoCloseable
,ChangedReadersObserver
,InternalScanner
,KeyValueScanner
,Shipper
- Direct Known Subclasses:
MobStoreScanner
,ReversedStoreScanner
@Private
public class StoreScanner
extends NonReversedNonLazyKeyValueScanner
implements KeyValueScanner, InternalScanner, ChangedReadersObserver
Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List<KeyValue>
for a single row.
The implementation is not thread safe. So there will be no race between next and close. The only exception is updateReaders, it will be called in the memstore flush thread to indicate that there is a flush.
-
Field Summary
Modifier and TypeFieldDescriptionprivate long
private boolean
private final long
private final ReentrantLock
private boolean
private final CellComparator
private long
(package private) final List<KeyValueScanner>
static final long
Default value ofHBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK
.private ExecutorService
private final boolean
private boolean
private final List<KeyValueScanner>
private final ReentrantLock
private final boolean
static final String
The number of cells scanned in between timeout checks.protected KeyValueHeap
private long
The number of KVs seen by the scanner.(package private) static final boolean
We don't ever expect to change this, the constant is just for clarity.private static boolean
Used during unit testing to ensure that lazy seek does save seek opsprivate static final org.slf4j.Logger
private ScanQueryMatcher
private final long
(package private) long
private final List<KeyValueScanner>
private final int
(package private) long
private final long
private final long
private boolean
A flag that enables StoreFileScanner parallel-seekingprivate final long
private ExtendedCell
protected final long
private final Scan.ReadType
private final Scan
private static final Scan
private final List<KeyValueScanner>
private boolean
protected final HStore
private int
private int
static final String
static final String
If the read type is Scan.ReadType.DEFAULT, we will start with pread, and if the kvs we scanned reaches this limit, we will reopen the scanner with stream.private boolean
private final boolean
Fields inherited from interface org.apache.hadoop.hbase.regionserver.KeyValueScanner
NO_NEXT_INDEXED_KEY
-
Constructor Summary
ModifierConstructorDescription(package private)
StoreScanner
(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns, List<? extends KeyValueScanner> scanners) (package private)
StoreScanner
(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns, List<? extends KeyValueScanner> scanners, ScanType scanType) private
StoreScanner
(HStore store, Scan scan, ScanInfo scanInfo, int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) An internal constructor.StoreScanner
(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) Used for compactions that drop deletes from a limited range of rows.StoreScanner
(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) Used for store file compaction and memstore compaction.private
StoreScanner
(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) StoreScanner
(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns, long readPt) Opens a scanner across memstore, snapshot, and all StoreFiles.(package private)
StoreScanner
(ScanInfo scanInfo, int maxVersions, ScanType scanType, List<? extends KeyValueScanner> scanners) StoreScanner
(ScanInfo scanInfo, ScanType scanType, List<? extends KeyValueScanner> scanners) -
Method Summary
Modifier and TypeMethodDescriptionprivate void
addCurrentScanners
(List<? extends KeyValueScanner> scanners) protected final boolean
protected void
checkScanOrder
(Cell prevKV, Cell kv, CellComparator comparator) Check whether scan as expected orderprivate static void
clearAndClose
(List<KeyValueScanner> scanners) void
close()
Close the KeyValue scanner.private void
close
(boolean withDelayedScannersClose) (package private) static void
enableLazySeekGlobally
(boolean enable) (package private) List<KeyValueScanner>
Used in testing.long
Returns The estimated number of KVs seen by this scanner (includes some skipped KVs).long
Returns the read point of the current scanprivate static boolean
isOnlyLatestVersionScan
(Scan scan) (package private) boolean
private ScannerContext.NextState
needToReturn
(List<? super ExtendedCell> outResult) If the top cell won't be flushed into disk, the new top cell may be changed after #reopenAfterFlush.protected KeyValueHeap
newKVHeap
(List<? extends KeyValueScanner> scanners, CellComparator comparator) next()
Return the next Cell in this scanner, iterating the scannerboolean
next
(List<? super ExtendedCell> outResult, ScannerContext scannerContext) Get the next row of values from this Store.private void
parallelSeek
(List<? extends KeyValueScanner> scanners, ExtendedCell kv) Seek storefiles in parallel to optimize IO latency as much as possiblepeek()
Look at the next Cell in this scanner, but do not iterate scanner.protected final boolean
Returns if top of heap has changed (and KeyValueHeap has to try the next KV)boolean
reseek
(ExtendedCell kv) Reseek the scanner at or after the specified KeyValue.protected void
resetKVHeap
(List<? extends KeyValueScanner> scanners, CellComparator comparator) private void
resetQueryMatcher
(ExtendedCell lastTopKey) boolean
seek
(ExtendedCell key) Seek the scanner at or after the specified KeyValue.private void
seekAllScanner
(ScanInfo scanInfo, List<? extends KeyValueScanner> scanners) protected boolean
Do a reseek in a normal StoreScanner(scan forward)private void
private void
protected void
seekScanners
(List<? extends KeyValueScanner> scanners, ExtendedCell seekKey, boolean isLazy, boolean isParallelSeek) Seek the specified scanners with the given keyprotected boolean
protected List<KeyValueScanner>
selectScannersFrom
(HStore store, List<? extends KeyValueScanner> allScanners) Filters the given list of scanners using Bloom filter, time range, and TTL.void
shipped()
Called after a batch of rows scanned and set to be returned to client.protected boolean
protected boolean
trySkipToNextRow
(ExtendedCell cell) See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).(package private) void
private void
updateMetricsStore
(boolean memstoreRead) void
updateReaders
(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners) Notify observers.Methods inherited from class org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner
backwardSeek, seekToLastRow, seekToPreviousRow
Methods inherited from class org.apache.hadoop.hbase.regionserver.NonLazyKeyValueScanner
doRealSeek, enforceSeek, getFilePath, isFileScanner, realSeekDone, recordBlockSize, requestSeek, shouldUseScanner
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
Methods inherited from interface org.apache.hadoop.hbase.regionserver.InternalScanner
next
Methods inherited from interface org.apache.hadoop.hbase.regionserver.KeyValueScanner
backwardSeek, enforceSeek, getFilePath, getScannerOrder, isFileScanner, realSeekDone, recordBlockSize, requestSeek, seekToLastRow, seekToPreviousRow, shouldUseScanner
-
Field Details
-
LOG
-
store
-
comparator
-
matcher
-
heap
-
cacheBlocks
-
countPerRow
-
storeLimit
-
storeOffset
-
closing
-
get
-
explicitColumnQuery
-
useRowColBloom
-
parallelSeekEnabled
A flag that enables StoreFileScanner parallel-seeking -
executor
-
scan
-
oldestUnexpiredTS
-
now
-
minVersions
-
maxRowSize
-
cellsPerHeartbeatCheck
-
memstoreOnlyReads
long memstoreOnlyReads -
mixedReads
long mixedReads -
scannersForDelayedClose
-
kvsScanned
The number of KVs seen by the scanner. Includes explicitly skipped KVs, but not KVs skipped via seeking to next row/column. TODO: estimate them? -
prevCell
-
preadMaxBytes
-
bytesRead
-
LAZY_SEEK_ENABLED_BY_DEFAULT
We don't ever expect to change this, the constant is just for clarity.- See Also:
-
STORESCANNER_PARALLEL_SEEK_ENABLE
- See Also:
-
lazySeekEnabledGlobally
Used during unit testing to ensure that lazy seek does save seek ops -
HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK
The number of cells scanned in between timeout checks. Specifying a larger value means that timeout checks will occur less frequently. Specifying a small value will lead to more frequent timeout checks.- See Also:
-
DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK
Default value ofHBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK
.- See Also:
-
STORESCANNER_PREAD_MAX_BYTES
If the read type is Scan.ReadType.DEFAULT, we will start with pread, and if the kvs we scanned reaches this limit, we will reopen the scanner with stream. The default value is 4 times of block size for this store. If configured with a value <0, for all scans with ReadType DEFAULT, we will open scanner with stream mode itself.- See Also:
-
readType
-
scanUsePread
-
flushed
-
flushedstoreFileScanners
-
memStoreScannersAfterFlush
-
currentScanners
-
flushLock
-
closeLock
-
readPt
-
topChanged
-
SCAN_FOR_COMPACTION
-
-
Constructor Details
-
StoreScanner
private StoreScanner(HStore store, Scan scan, ScanInfo scanInfo, int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) An internal constructor. -
StoreScanner
public StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException Used for store file compaction and memstore compaction.Opens a scanner across specified StoreFiles/MemStoreSegments.
- Parameters:
store
- who we scanscanners
- ancillary scannerssmallestReadPoint
- the readPoint that we should use for tracking versions- Throws:
IOException
-
StoreScanner
public StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException Used for compactions that drop deletes from a limited range of rows.Opens a scanner across specified StoreFiles.
- Parameters:
store
- who we scanscanners
- ancillary scannerssmallestReadPoint
- the readPoint that we should use for tracking versionsdropDeletesFromRow
- The inclusive left bound of the range; can be EMPTY_START_ROW.dropDeletesToRow
- The exclusive right bound of the range; can be EMPTY_END_ROW.- Throws:
IOException
-
StoreScanner
private StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException - Throws:
IOException
-
StoreScanner
public StoreScanner(ScanInfo scanInfo, ScanType scanType, List<? extends KeyValueScanner> scanners) throws IOException - Throws:
IOException
-
StoreScanner
StoreScanner(ScanInfo scanInfo, int maxVersions, ScanType scanType, List<? extends KeyValueScanner> scanners) throws IOException - Throws:
IOException
-
-
Method Details
-
addCurrentScanners
-
isOnlyLatestVersionScan
-
seekAllScanner
private void seekAllScanner(ScanInfo scanInfo, List<? extends KeyValueScanner> scanners) throws IOException - Throws:
IOException
-
isScanUsePread
boolean isScanUsePread() -
seekScanners
protected void seekScanners(List<? extends KeyValueScanner> scanners, ExtendedCell seekKey, boolean isLazy, boolean isParallelSeek) throws IOException Seek the specified scanners with the given key- Parameters:
isLazy
- true if using lazy seekisParallelSeek
- true if using parallel seek- Throws:
IOException
-
resetKVHeap
protected void resetKVHeap(List<? extends KeyValueScanner> scanners, CellComparator comparator) throws IOException - Throws:
IOException
-
newKVHeap
protected KeyValueHeap newKVHeap(List<? extends KeyValueScanner> scanners, CellComparator comparator) throws IOException - Throws:
IOException
-
selectScannersFrom
protected List<KeyValueScanner> selectScannersFrom(HStore store, List<? extends KeyValueScanner> allScanners) Filters the given list of scanners using Bloom filter, time range, and TTL.Will be overridden by testcase so declared as protected.
-
peek
Description copied from interface:KeyValueScanner
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.- Specified by:
peek
in interfaceKeyValueScanner
- Returns:
- the next Cell
-
next
Description copied from interface:KeyValueScanner
Return the next Cell in this scanner, iterating the scanner- Specified by:
next
in interfaceKeyValueScanner
- Returns:
- the next Cell
-
close
Description copied from interface:KeyValueScanner
Close the KeyValue scanner.- Specified by:
close
in interfaceAutoCloseable
- Specified by:
close
in interfaceCloseable
- Specified by:
close
in interfaceInternalScanner
- Specified by:
close
in interfaceKeyValueScanner
-
close
-
seek
Description copied from interface:KeyValueScanner
Seek the scanner at or after the specified KeyValue.- Specified by:
seek
in interfaceKeyValueScanner
- Parameters:
key
- seek value- Returns:
- true if scanner has values left, false if end of scanner
- Throws:
IOException
-
next
public boolean next(List<? super ExtendedCell> outResult, ScannerContext scannerContext) throws IOException Get the next row of values from this Store.- Specified by:
next
in interfaceInternalScanner
- Parameters:
outResult
- return output array. We will only add ExtendedCell to this list, but for CP users, you'd better just useRawCell
asExtendedCell
is IA.Private.- Returns:
- true if there are more rows, false if scanner is done
- Throws:
IOException
-
updateMetricsStore
-
needToReturn
If the top cell won't be flushed into disk, the new top cell may be changed after #reopenAfterFlush. Because the older top cell only exist in the memstore scanner but the memstore scanner is replaced by hfile scanner after #reopenAfterFlush. If the row of top cell is changed, we should return the current cells. Otherwise, we may return the cells across different rows.- Parameters:
outResult
- the cells which are visible for user scan- Returns:
- null is the top cell doesn't change. Otherwise, the NextState to return
-
seekOrSkipToNextRow
- Throws:
IOException
-
seekOrSkipToNextColumn
- Throws:
IOException
-
trySkipToNextRow
See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109). ScanQueryMatcher may issue SEEK hints, such as seek to next column, next row, or seek to an arbitrary seek key. This method decides whether a seek is the most efficient _actual_ way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP, SKIP inside the current, loaded block). It does this by looking at the next indexed key of the current HFile. This key is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with the current Cell but compare as though it were a seek key; see down in matcher.compareKeyForNextRow, etc). If the compare gets us onto the next block we *_SEEK, otherwise we just SKIP to the next requested cell.Other notes:
- Rows can straddle block boundaries
- Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a different block than column C1 at T2)
- We want to SKIP if the chance is high that we'll find the desired Cell after a few SKIPs...
- We want to SEEK when the chance is high that we'll be able to seek past many Cells, especially if we know we need to go to the next block.
A good proxy (best effort) to determine whether SKIP is better than SEEK is whether we'll likely end up seeking to the next block (or past the next block) to get our next column. Example:
| BLOCK 1 | BLOCK 2 | | r1/c1, r1/c2, r1/c3 | r1/c4, r1/c5, r2/c1 | ^ ^ | | Next Index Key SEEK_NEXT_ROW (before r2/c1) | BLOCK 1 | BLOCK 2 | | r1/c1/t5, r1/c1/t4, r1/c1/t3 | r1/c1/t2, r1/c1/T1, r1/c2/T3 | ^ ^ | | Next Index Key SEEK_NEXT_COL
Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4 is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios where the SEEK will not land us in the next block, it is very likely better to issues a series of SKIPs.- Parameters:
cell
- current cell- Returns:
- true means skip to next row, false means not
- Throws:
IOException
-
trySkipToNextColumn
- Parameters:
cell
- current cell- Returns:
- true means skip to next column, false means not
- Throws:
IOException
-
getReadPoint
Description copied from interface:ChangedReadersObserver
Returns the read point of the current scan- Specified by:
getReadPoint
in interfaceChangedReadersObserver
-
clearAndClose
-
updateReaders
public void updateReaders(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException Description copied from interface:ChangedReadersObserver
Notify observers.
NOTE:Before we invoke this method,HStoreFile.increaseRefCount()
is invoked for everyHStoreFile
in 'sfs' input parameter to preventHStoreFile
is archived after a concurrent compaction, and after this method is invoked,HStoreFile.decreaseRefCount()
is invoked.So if you open theStoreFileReader
orStoreFileScanner
asynchronously in this method,you may need to invokeHStoreFile.increaseRefCount()
orHStoreFile.decreaseRefCount()
by yourself to prevent theHStoreFile
s be archived.- Specified by:
updateReaders
in interfaceChangedReadersObserver
- Parameters:
sfs
- The new filesmemStoreScanners
- scanner of current memstore- Throws:
IOException
- e
-
reopenAfterFlush
Returns if top of heap has changed (and KeyValueHeap has to try the next KV)- Throws:
IOException
-
resetQueryMatcher
-
checkScanOrder
Check whether scan as expected order- Throws:
IOException
-
seekToNextRow
- Throws:
IOException
-
seekAsDirection
Do a reseek in a normal StoreScanner(scan forward)- Returns:
- true if scanner has values left, false if end of scanner
- Throws:
IOException
-
reseek
Description copied from interface:KeyValueScanner
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.- Specified by:
reseek
in interfaceKeyValueScanner
- Parameters:
kv
- seek value (should be non-null)- Returns:
- true if scanner has values left, false if end of scanner
- Throws:
IOException
-
trySwitchToStreamRead
void trySwitchToStreamRead() -
checkFlushed
-
parallelSeek
private void parallelSeek(List<? extends KeyValueScanner> scanners, ExtendedCell kv) throws IOException Seek storefiles in parallel to optimize IO latency as much as possible- Parameters:
scanners
- the listKeyValueScanner
s to be read fromkv
- the KeyValue on which the operation is being requested- Throws:
IOException
-
getAllScannersForTesting
Used in testing.- Returns:
- all scanners in no particular order
-
enableLazySeekGlobally
-
getEstimatedNumberOfKvsScanned
Returns The estimated number of KVs seen by this scanner (includes some skipped KVs). -
getNextIndexedKey
- Specified by:
getNextIndexedKey
in interfaceKeyValueScanner
- Overrides:
getNextIndexedKey
in classNonLazyKeyValueScanner
- 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.
-
shipped
Description copied from interface:Shipper
Called after a batch of rows scanned and set to be returned to client. Any in between cleanup can be done here.- Specified by:
shipped
in interfaceShipper
- Overrides:
shipped
in classNonLazyKeyValueScanner
- Throws:
IOException
-