Interface KeyValueScanner

All Superinterfaces:
AutoCloseable, Closeable, Shipper
All Known Implementing Classes:
CollectionBackedScanner, KeyValueHeap, MobStoreScanner, NonLazyKeyValueScanner, NonReversedNonLazyKeyValueScanner, ReversedKeyValueHeap, ReversedMobStoreScanner, ReversedStoreScanner, SegmentScanner, SnapshotSegmentScanner, StoreFileScanner, StoreScanner

@Private public interface KeyValueScanner extends Shipper, Closeable
Scanner that returns the next KeyValue.
  • Field Summary

    Fields
    Modifier and Type
    Field
    Description
    static 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 Type
    Method
    Description
    boolean
    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 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.
    Return the next Cell in this scanner, iterating the scanner
    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 to seek(org.apache.hadoop.hbase.ExtendedCell) (or reseek(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 the scanner at or after the specified KeyValue.
    boolean
    Seek the scanner at or after the specified KeyValue.
    boolean
    Seek the scanner at the first KeyValue of last row
    boolean
    Seek the scanner at the first Cell of the row which is the previous row of specified key
    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.

    Methods inherited from interface org.apache.hadoop.hbase.regionserver.Shipper

    shipped
  • 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

      boolean seek(ExtendedCell key) throws IOException
      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

      boolean reseek(ExtendedCell key) throws IOException
      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

      default long 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 interface AutoCloseable
      Specified by:
      close in interface Closeable
    • shouldUseScanner

      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.
      Parameters:
      scan - the scan that we are selecting scanners for
      store - 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

      boolean requestSeek(ExtendedCell kv, boolean forward, boolean useBloom) throws IOException
      Similar to seek(org.apache.hadoop.hbase.ExtendedCell) (or reseek(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 seek
      useBloom - 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

      void enforceSeek() throws IOException
      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 call realSeekDone() first.
      Throws:
      IOException
    • isFileScanner

      boolean isFileScanner()
      Returns true if this is a file scanner. Otherwise a memory scanner is assumed.
    • recordBlockSize

      void recordBlockSize(IntConsumer blockSizeConsumer)
      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

      boolean backwardSeek(ExtendedCell key) throws IOException
      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

      boolean seekToLastRow() throws IOException
      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.