Class Get

All Implemented Interfaces:
Comparable<Row>, Attributes, Row

@Public public class Get extends Query implements Row
Used to perform Get operations on a single row.

To get everything for a row, instantiate a Get object with the row to get. To further narrow the scope of what to Get, use the methods below.

To get all columns from specific families, execute addFamily for each family to retrieve.

To get specific columns, execute addColumn for each column to retrieve.

To only retrieve columns within a specific range of version timestamps, execute setTimeRange.

To only retrieve columns with a specific timestamp, execute setTimestamp.

To limit the number of versions of each column to be returned, execute setMaxVersions.

To add a filter, call setFilter.

  • Field Details

  • Constructor Details

    • Get

      public Get(byte[] row)
      Create a Get operation for the specified row.

      If no further operations are done, this will get the latest version of all columns in all families of the specified row.

      Parameters:
      row - row key
    • Get

      public Get(Get get)
      Copy-constructor
    • Get

      public Get(byte[] row, int rowOffset, int rowLength)
      Create a Get operation for the specified row.
    • Get

      public Get(ByteBuffer row)
      Create a Get operation for the specified row.
  • Method Details

    • isCheckExistenceOnly

      public boolean isCheckExistenceOnly()
    • setCheckExistenceOnly

      public Get setCheckExistenceOnly(boolean checkExistenceOnly)
    • isClosestRowBefore

      @Deprecated public boolean isClosestRowBefore()
      Deprecated.
      since 2.0.0 and will be removed in 3.0.0
      This will always return the default value which is false as client cannot set the value to this property any more.
    • setClosestRowBefore

      @Deprecated public Get setClosestRowBefore(boolean closestRowBefore)
      Deprecated.
      since 2.0.0 and will be removed in 3.0.0
      This is not used any more and does nothing. Use reverse scan instead.
    • addFamily

      public Get addFamily(byte[] family)
      Get all columns from the specified family.

      Overrides previous calls to addColumn for this family.

      Parameters:
      family - family name
      Returns:
      the Get object
    • addColumn

      public Get addColumn(byte[] family, byte[] qualifier)
      Get the column from the specific family with the specified qualifier.

      Overrides previous calls to addFamily for this family.

      Parameters:
      family - family name
      qualifier - column qualifier
      Returns:
      the Get objec
    • setTimeRange

      public Get setTimeRange(long minStamp, long maxStamp) throws IOException
      Get versions of columns only within the specified timestamp range, [minStamp, maxStamp).
      Parameters:
      minStamp - minimum timestamp value, inclusive
      maxStamp - maximum timestamp value, exclusive
      Returns:
      this for invocation chaining
      Throws:
      IOException
    • setTimeStamp

      @Deprecated public Get setTimeStamp(long timestamp) throws IOException
      Deprecated.
      As of release 2.0.0, this will be removed in HBase 3.0.0. Use setTimestamp(long) instead
      Get versions of columns with the specified timestamp.
      Parameters:
      timestamp - version timestamp
      Returns:
      this for invocation chaining
      Throws:
      IOException
    • setTimestamp

      public Get setTimestamp(long timestamp)
      Get versions of columns with the specified timestamp.
      Parameters:
      timestamp - version timestamp
      Returns:
      this for invocation chaining
    • setColumnFamilyTimeRange

      public Get setColumnFamilyTimeRange(byte[] cf, long minStamp, long maxStamp)
      Description copied from class: Query
      Get versions of columns only within the specified timestamp range, [minStamp, maxStamp) on a per CF bases. Note, default maximum versions to return is 1. If your time range spans more than one version and you want all versions returned, up the number of versions beyond the default. Column Family time ranges take precedence over the global time range.
      Overrides:
      setColumnFamilyTimeRange in class Query
      Parameters:
      cf - the column family for which you want to restrict
      minStamp - minimum timestamp value, inclusive
      maxStamp - maximum timestamp value, exclusive
    • setMaxVersions

      Deprecated.
      It is easy to misunderstand with column family's max versions, so use readAllVersions() instead.
      Get all available versions.
      Returns:
      this for invocation chaining
    • setMaxVersions

      @Deprecated public Get setMaxVersions(int maxVersions) throws IOException
      Deprecated.
      It is easy to misunderstand with column family's max versions, so use readVersions(int) instead.
      Get up to the specified number of versions of each column.
      Parameters:
      maxVersions - maximum versions for each column
      Returns:
      this for invocation chaining
      Throws:
      IOException - if invalid number of versions
    • readAllVersions

      public Get readAllVersions()
      Get all available versions.
      Returns:
      this for invocation chaining
    • readVersions

      public Get readVersions(int versions) throws IOException
      Get up to the specified number of versions of each column.
      Parameters:
      versions - specified number of versions for each column
      Returns:
      this for invocation chaining
      Throws:
      IOException - if invalid number of versions
    • setLoadColumnFamiliesOnDemand

      public Get setLoadColumnFamiliesOnDemand(boolean value)
      Description copied from class: Query
      Set the value indicating whether loading CFs on demand should be allowed (cluster default is false). On-demand CF loading doesn't load column families until necessary, e.g. if you filter on one column, the other column family data will be loaded only for the rows that are included in result, not all rows like in normal case. With column-specific filters, like SingleColumnValueFilter w/filterIfMissing == true, this can deliver huge perf gains when there's a cf with lots of data; however, it can also lead to some inconsistent results, as follows: - if someone does a concurrent update to both column families in question you may get a row that never existed, e.g. for { rowKey = 5, { cat_videos => 1 }, { video => "my cat" } } someone puts rowKey 5 with { cat_videos => 0 }, { video => "my dog" }, concurrent scan filtering on "cat_videos == 1" can get { rowKey = 5, { cat_videos => 1 }, { video => "my dog" } }. - if there's a concurrent split and you have more than 2 column families, some rows may be missing some column families.
      Overrides:
      setLoadColumnFamiliesOnDemand in class Query
    • setMaxResultsPerColumnFamily

      public Get setMaxResultsPerColumnFamily(int limit)
      Set the maximum number of values to return per row per Column Family
      Parameters:
      limit - the maximum number of values returned / row / CF
      Returns:
      this for invocation chaining
    • setRowOffsetPerColumnFamily

      public Get setRowOffsetPerColumnFamily(int offset)
      Set offset for the row per Column Family. This offset is only within a particular row/CF combination. It gets reset back to zero when we move to the next row or CF.
      Parameters:
      offset - is the number of kvs that will be skipped.
      Returns:
      this for invocation chaining
    • setFilter

      public Get setFilter(Filter filter)
      Description copied from class: Query
      Apply the specified server-side filter when performing the Query. Only Filter.filterCell(org.apache.hadoop.hbase.Cell) is called AFTER all tests for ttl, column match, deletes and column family's max versions have been run.
      Overrides:
      setFilter in class Query
      Parameters:
      filter - filter to run on the server
      Returns:
      this for invocation chaining
    • setCacheBlocks

      public Get setCacheBlocks(boolean cacheBlocks)
      Set whether blocks should be cached for this Get.

      This is true by default. When true, default settings of the table and family are used (this will never override caching blocks if the block cache is disabled for that family or entirely).

      Parameters:
      cacheBlocks - if false, default settings are overridden and blocks will not be cached
    • getCacheBlocks

      public boolean getCacheBlocks()
      Get whether blocks should be cached for this Get.
      Returns:
      true if default caching should be used, false if blocks should not be cached
    • getRow

      public byte[] getRow()
      Method for retrieving the get's row
      Specified by:
      getRow in interface Row
    • getMaxVersions

      public int getMaxVersions()
      Method for retrieving the get's maximum number of version
      Returns:
      the maximum number of version to fetch for this get
    • getMaxResultsPerColumnFamily

      Method for retrieving the get's maximum number of values to return per Column Family
      Returns:
      the maximum number of values to fetch per CF
    • getRowOffsetPerColumnFamily

      Method for retrieving the get's offset per row per column family (#kvs to be skipped)
      Returns:
      the row offset
    • getTimeRange

      Method for retrieving the get's TimeRange
    • familySet

      public Set<byte[]> familySet()
      Method for retrieving the keys in the familyMap
      Returns:
      keys in the current familyMap
    • numFamilies

      public int numFamilies()
      Method for retrieving the number of families to get from
      Returns:
      number of families
    • hasFamilies

      public boolean hasFamilies()
      Method for checking if any families have been inserted into this Get
      Returns:
      true if familyMap is non empty false otherwise
    • getFamilyMap

      public Map<byte[],NavigableSet<byte[]>> getFamilyMap()
      Method for retrieving the get's familyMap
    • getFingerprint

      Compile the table and column family (i.e. schema) information into a String. Useful for parsing and aggregation by debugging, logging, and administration tools.
      Specified by:
      getFingerprint in class Operation
      Returns:
      a map containing fingerprint information (i.e. column families)
    • toMap

      public Map<String,Object> toMap(int maxCols)
      Compile the details beyond the scope of getFingerprint (row, columns, timestamps, etc.) into a Map along with the fingerprinted information. Useful for debugging, logging, and administration tools.
      Specified by:
      toMap in class Operation
      Parameters:
      maxCols - a limit on the number of columns output prior to truncation
      Returns:
      a map containing parameters of a query (i.e. rows, columns...)
    • compareTo

      public int compareTo(Row other)
      Description copied from interface: Row
      Compare this row to another row.
      Specified by:
      compareTo in interface Comparable<Row>
      Specified by:
      compareTo in interface Row
    • hashCode

      public int hashCode()
      Overrides:
      hashCode in class Object
    • equals

      public boolean equals(Object obj)
      Overrides:
      equals in class Object
    • setAttribute

      public Get setAttribute(String name, byte[] value)
      Description copied from interface: Attributes
      Sets an attribute. In case value = null attribute is removed from the attributes map. Attribute names starting with _ indicate system attributes.
      Specified by:
      setAttribute in interface Attributes
      Overrides:
      setAttribute in class OperationWithAttributes
      Parameters:
      name - attribute name
      value - attribute value
    • setId

      public Get setId(String id)
      Description copied from class: OperationWithAttributes
      This method allows you to set an identifier on an operation. The original motivation for this was to allow the identifier to be used in slow query logging, but this could obviously be useful in other places. One use of this could be to put a class.method identifier in here to see where the slow query is coming from. id to set for the scan
      Overrides:
      setId in class OperationWithAttributes
    • setAuthorizations

      public Get setAuthorizations(Authorizations authorizations)
      Description copied from class: Query
      Sets the authorizations to be used by this Query
      Overrides:
      setAuthorizations in class Query
    • setACL

      public Get setACL(Map<String,Permission> perms)
      Description copied from class: Query
      Set the ACL for the operation.
      Overrides:
      setACL in class Query
      Parameters:
      perms - A map of permissions for a user or users
    • setACL

      public Get setACL(String user, Permission perms)
      Description copied from class: Query
      Set the ACL for the operation.
      Overrides:
      setACL in class Query
      Parameters:
      user - User short name
      perms - Permissions for the user
    • setConsistency

      public Get setConsistency(Consistency consistency)
      Description copied from class: Query
      Sets the consistency level for this operation
      Overrides:
      setConsistency in class Query
      Parameters:
      consistency - the consistency level
    • setReplicaId

      public Get setReplicaId(int Id)
      Description copied from class: Query
      Specify region replica id where Query will fetch data from. Use this together with Query.setConsistency(Consistency) passing Consistency.TIMELINE to read data from a specific replicaId.
      Expert: This is an advanced API exposed. Only use it if you know what you are doing
      Overrides:
      setReplicaId in class Query
    • setIsolationLevel

      Description copied from class: Query
      Set the isolation level for this query. If the isolation level is set to READ_UNCOMMITTED, then this query will return data from committed and uncommitted transactions. If the isolation level is set to READ_COMMITTED, then this query will return data from committed transactions only. If a isolation level is not explicitly set on a Query, then it is assumed to be READ_COMMITTED.
      Overrides:
      setIsolationLevel in class Query
      Parameters:
      level - IsolationLevel for this query
    • setPriority

      public Get setPriority(int priority)
      Overrides:
      setPriority in class OperationWithAttributes