Class PrivateCellUtil

java.lang.Object
org.apache.hadoop.hbase.PrivateCellUtil

@Private public final class PrivateCellUtil extends Object
Utility methods helpful slinging Cell instances. It has more powerful and rich set of APIs than those in CellUtil for internal usage.
  • Field Details

  • Constructor Details

    • PrivateCellUtil

      private PrivateCellUtil()
      Private constructor to keep this class from being instantiated.
  • Method Details

    • fillRowRange

      public static ByteRange fillRowRange(Cell cell, ByteRange range)
      ByteRange
    • fillFamilyRange

      public static ByteRange fillFamilyRange(Cell cell, ByteRange range)
    • fillQualifierRange

      public static ByteRange fillQualifierRange(Cell cell, ByteRange range)
    • fillValueRange

      public static ByteRange fillValueRange(Cell cell, ByteRange range)
    • fillTagRange

      public static ByteRange fillTagRange(ExtendedCell cell, ByteRange range)
    • getRowByte

      public static byte getRowByte(Cell cell, int index)
      misc
    • getQualifierByte

      public static byte getQualifierByte(Cell cell, int index)
    • getValueBufferShallowCopy

    • createCell

      public static ExtendedCell createCell(ExtendedCell cell, List<Tag> tags)
      Returns A new cell which is having the extra tags also added to it.
    • createCell

      public static ExtendedCell createCell(ExtendedCell cell, byte[] tags)
      Returns A new cell which is having the extra tags also added to it.
    • createCell

      public static ExtendedCell createCell(ExtendedCell cell, byte[] value, byte[] tags)
    • matchingRows

      public static boolean matchingRows(Cell left, byte[] buf, int offset, int length)
    • matchingFamily

      public static boolean matchingFamily(Cell left, byte[] buf, int offset, int length)
    • matchingQualifier

      public static boolean matchingQualifier(Cell left, byte[] buf, int offset, int length)
      Finds if the qualifier part of the cell and the KV serialized byte[] are equal
      Parameters:
      left - the cell with which we need to match the qualifier
      buf - the serialized keyvalue format byte[]
      offset - the offset of the qualifier in the byte[]
      length - the length of the qualifier in the byte[]
      Returns:
      true if the qualifier matches, false otherwise
    • qualifierStartsWith

      public static boolean qualifierStartsWith(Cell left, byte[] startsWith)
      Finds if the start of the qualifier part of the Cell matches buf
      Parameters:
      left - the cell with which we need to match the qualifier
      startsWith - the serialized keyvalue format byte[]
      Returns:
      true if the qualifier have same staring characters, false otherwise
    • matchingColumn

      public static boolean matchingColumn(Cell left, byte[] fam, int foffset, int flength, byte[] qual, int qoffset, int qlength)
    • matchingValue

      public static boolean matchingValue(Cell left, Cell right, int lvlength, int rvlength)
    • matchingType

      public static boolean matchingType(ExtendedCell a, ExtendedCell b)
    • matchingTags

      public static boolean matchingTags(ExtendedCell left, ExtendedCell right)
    • matchingTags

      public static boolean matchingTags(ExtendedCell left, ExtendedCell right, int llength, int rlength)
    • isDelete

      public static boolean isDelete(byte type)
      Return true if a delete type, a KeyValue.Type.Delete or a {KeyValue.Type#DeleteFamily} or a KeyValue.Type.DeleteColumn KeyValue type.
    • isDeleteType

      public static boolean isDeleteType(ExtendedCell cell)
      Returns True if this cell is a KeyValue.Type.Delete type.
    • isDeleteFamily

      public static boolean isDeleteFamily(ExtendedCell cell)
    • isDeleteFamilyVersion

      public static boolean isDeleteFamilyVersion(ExtendedCell cell)
    • isDeleteColumns

      public static boolean isDeleteColumns(ExtendedCell cell)
    • isDeleteColumnVersion

      public static boolean isDeleteColumnVersion(ExtendedCell cell)
    • isDeleteColumnOrFamily

      public static boolean isDeleteColumnOrFamily(ExtendedCell cell)
      Returns True if this cell is a delete family or column type.
    • cloneTags

      public static byte[] cloneTags(ExtendedCell cell)
    • copyTagsTo

      public static int copyTagsTo(ExtendedCell cell, byte[] destination, int destinationOffset)
      Copies the tags info into the tag portion of the cell
    • copyTagsTo

      public static int copyTagsTo(ExtendedCell cell, ByteBuffer destination, int destinationOffset)
      Copies the tags info into the tag portion of the cell
    • getTags

      public static List<Tag> getTags(ExtendedCell cell)
      Return tags in the given Cell as a List
      Parameters:
      cell - The Cell
      Returns:
      Tags in the given Cell as a List
    • getTag

      public static Optional<Tag> getTag(ExtendedCell cell, byte type)
      Retrieve Cell's first tag, matching the passed in type
      Parameters:
      cell - The Cell
      type - Type of the Tag to retrieve
      Returns:
      Optional, empty if there is no tag of the passed in tag type
    • tagsIterator

      public static Iterator<Tag> tagsIterator(ExtendedCell cell)
      Utility method to iterate through the tags in the given cell.
      Parameters:
      cell - The Cell over which tags iterator is needed.
      Returns:
      iterator for the tags
    • tagsIterator

      public static Iterator<Tag> tagsIterator(ByteBuffer tags, int offset, int length)
    • overlappingKeys

      public static boolean overlappingKeys(byte[] start1, byte[] end1, byte[] start2, byte[] end2)
      Returns true if the first range start1...end1 overlaps with the second range start2...end2, assuming the byte arrays represent row keys
    • writeRowKeyExcludingCommon

      public static void writeRowKeyExcludingCommon(Cell cell, short rLen, int commonPrefix, DataOutputStream out) throws IOException
      Write rowkey excluding the common part.
      Throws:
      IOException
    • writeRowSkippingBytes

      public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength, int commonPrefix) throws IOException
      Writes the row from the given cell to the output stream excluding the common prefix
      Parameters:
      out - The dataoutputstream to which the data has to be written
      cell - The cell whose contents has to be written
      rlength - the row length
      Throws:
      IOException
    • findCommonPrefixInFlatKey

      public static int findCommonPrefixInFlatKey(ExtendedCell c1, ExtendedCell c2, boolean bypassFamilyCheck, boolean withTsType)
      Find length of common prefix in keys of the cells, considering key as byte[] if serialized in KeyValue. The key format is <2 bytes rk len><rk><1 byte cf len><cf><qualifier><8 bytes timestamp><1 byte type>
      Parameters:
      c1 - the cell
      c2 - the cell
      bypassFamilyCheck - when true assume the family bytes same in both cells. Pass it as true when dealing with Cells in same CF so as to avoid some checks
      withTsType - when true check timestamp and type bytes also.
      Returns:
      length of common prefix
    • compareKeyBasedOnColHint

      public static final int compareKeyBasedOnColHint(CellComparator comparator, ExtendedCell nextIndexedCell, ExtendedCell currentCell, int foff, int flen, byte[] colHint, int coff, int clen, long ts, byte type)
      Used to compare two cells based on the column hint provided. This is specifically used when we need to optimize the seeks based on the next indexed key. This is an advanced usage API specifically needed for some optimizations.
      Parameters:
      nextIndexedCell - the next indexed cell
      currentCell - the cell to be compared
      foff - the family offset of the currentCell
      flen - the family length of the currentCell
      colHint - the column hint provided - could be null
      coff - the offset of the column hint if provided, if not offset of the currentCell's qualifier
      clen - the length of the column hint if provided, if not length of the currentCell's qualifier
      ts - the timestamp to be seeked
      type - the type to be seeked
      Returns:
      an int based on the given column hint TODO : To be moved out of here because this is a special API used in scan optimization.
    • compareKeyIgnoresMvcc

      public static final int compareKeyIgnoresMvcc(CellComparator comparator, Cell left, Cell right)
      Compares only the key portion of a cell. It does not include the sequence id/mvcc of the cell
      Returns:
      an int greater than 0 if left > than right lesser than 0 if left < than right equal to 0 if left is equal to right
    • compareRow

      public static int compareRow(Cell cell, ByteArrayComparable comparator)
      Compare cell's row against given comparator
      Parameters:
      cell - the cell to use for comparison
      comparator - the CellComparator to use for comparison
      Returns:
      result comparing cell's row
    • compareFamily

      public static int compareFamily(Cell cell, ByteArrayComparable comparator)
      Compare cell's column family against given comparator
      Parameters:
      cell - the cell to use for comparison
      comparator - the CellComparator to use for comparison
      Returns:
      result comparing cell's column family
    • compareQualifier

      public static int compareQualifier(Cell cell, ByteArrayComparable comparator)
      Compare cell's qualifier against given comparator
      Parameters:
      cell - the cell to use for comparison
      comparator - the CellComparator to use for comparison
      Returns:
      result comparing cell's qualifier
    • toType

      public static Cell.Type toType(byte type)
    • toTypeByte

      public static KeyValue.Type toTypeByte(Cell.Type type)
    • compareValue

      public static int compareValue(Cell cell, ByteArrayComparable comparator)
      Compare cell's value against given comparator
      Parameters:
      cell - the cell to use for comparison
      comparator - the CellComparator to use for comparison
      Returns:
      result comparing cell's value
    • writeFlatKey

      public static void writeFlatKey(ExtendedCell cell, DataOutput out) throws IOException
      Writes the Cell's key part as it would have serialized in a KeyValue. The format is <2 bytes rk len><rk><1 byte cf len><cf><qualifier><8 bytes timestamp><1 byte type>
      Throws:
      IOException
    • writeFlatKey

      public static int writeFlatKey(ExtendedCell cell, OutputStream out) throws IOException
      Throws:
      IOException
    • setSequenceId

      public static void setSequenceId(Cell cell, long seqId) throws IOException
      Sets the given seqId to the cell. Marked as audience Private as of 1.2.0. Setting a Cell sequenceid is an internal implementation detail not for general public use.
      Throws:
      IOException - when the passed cell is not of type ExtendedCell
    • setTimestamp

      public static void setTimestamp(Cell cell, long ts) throws IOException
      Sets the given timestamp to the cell.
      Throws:
      IOException - when the passed cell is not of type ExtendedCell
    • setTimestamp

      public static void setTimestamp(Cell cell, byte[] ts) throws IOException
      Sets the given timestamp to the cell.
      Throws:
      IOException - when the passed cell is not of type ExtendedCell
    • updateLatestStamp

      public static boolean updateLatestStamp(ExtendedCell cell, long ts) throws IOException
      Sets the given timestamp to the cell iff current timestamp is HConstants.LATEST_TIMESTAMP.
      Returns:
      True if cell timestamp is modified.
      Throws:
      IOException - when the passed cell is not of type ExtendedCell
    • updateLatestStamp

      public static boolean updateLatestStamp(Cell cell, byte[] ts) throws IOException
      Sets the given timestamp to the cell iff current timestamp is HConstants.LATEST_TIMESTAMP.
      Returns:
      True if cell timestamp is modified.
      Throws:
      IOException - when the passed cell is not of type ExtendedCell
    • writeRow

      public static void writeRow(OutputStream out, Cell cell, short rlength) throws IOException
      Writes the row from the given cell to the output stream
      Parameters:
      out - The outputstream to which the data has to be written
      cell - The cell whose contents has to be written
      rlength - the row length
      Throws:
      IOException
    • writeFamily

      public static void writeFamily(OutputStream out, Cell cell, byte flength) throws IOException
      Writes the family from the given cell to the output stream
      Parameters:
      out - The outputstream to which the data has to be written
      cell - The cell whose contents has to be written
      flength - the family length
      Throws:
      IOException
    • writeQualifier

      public static void writeQualifier(OutputStream out, Cell cell, int qlength) throws IOException
      Writes the qualifier from the given cell to the output stream
      Parameters:
      out - The outputstream to which the data has to be written
      cell - The cell whose contents has to be written
      qlength - the qualifier length
      Throws:
      IOException
    • writeQualifierSkippingBytes

      public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell, int qlength, int commonPrefix) throws IOException
      Writes the qualifier from the given cell to the output stream excluding the common prefix
      Parameters:
      out - The dataoutputstream to which the data has to be written
      cell - The cell whose contents has to be written
      qlength - the qualifier length
      Throws:
      IOException
    • writeValue

      public static void writeValue(OutputStream out, ExtendedCell cell, int vlength) throws IOException
      Writes the value from the given cell to the output stream
      Parameters:
      out - The outputstream to which the data has to be written
      cell - The cell whose contents has to be written
      vlength - the value length
      Throws:
      IOException
    • writeTags

      public static void writeTags(OutputStream out, ExtendedCell cell, int tagsLength) throws IOException
      Writes the tag from the given cell to the output stream
      Parameters:
      out - The outputstream to which the data has to be written
      cell - The cell whose contents has to be written
      tagsLength - the tag length
      Throws:
      IOException
    • equalsIgnoreMvccVersion

      public static boolean equalsIgnoreMvccVersion(ExtendedCell a, ExtendedCell b)
      special case for Cell.equals
    • getRowAsInt

      public static int getRowAsInt(Cell cell)
      Converts the rowkey bytes of the given cell into an int value
      Returns:
      rowkey as int
    • getValueAsLong

      public static long getValueAsLong(Cell cell)
      Converts the value bytes of the given cell into a long value
      Returns:
      value as long
    • getValueAsInt

      public static int getValueAsInt(Cell cell)
      Converts the value bytes of the given cell into a int value
      Returns:
      value as int
    • getValueAsDouble

      public static double getValueAsDouble(Cell cell)
      Converts the value bytes of the given cell into a double value
      Returns:
      value as double
    • getValueAsBigDecimal

      public static BigDecimal getValueAsBigDecimal(Cell cell)
      Converts the value bytes of the given cell into a BigDecimal
      Returns:
      value as BigDecimal
    • compressTags

      public static void compressTags(OutputStream out, ExtendedCell cell, TagCompressionContext tagCompressionContext) throws IOException
      Compresses the tags to the given outputstream using the TagcompressionContext
      Parameters:
      out - the outputstream to which the compression should happen
      cell - the cell which has tags
      tagCompressionContext - the TagCompressionContext
      Throws:
      IOException - can throw IOException if the compression encounters issue
    • compressRow

      public static void compressRow(OutputStream out, Cell cell, Dictionary dict) throws IOException
      Throws:
      IOException
    • compressFamily

      public static void compressFamily(OutputStream out, Cell cell, Dictionary dict) throws IOException
      Throws:
      IOException
    • compressQualifier

      public static void compressQualifier(OutputStream out, Cell cell, Dictionary dict) throws IOException
      Throws:
      IOException
    • compare

      public static final int compare(CellComparator comparator, ExtendedCell left, byte[] key, int offset, int length)
      Used when a cell needs to be compared with a key byte[] such as cases of finding the index from the index block, bloom keys from the bloom blocks This byte[] is expected to be serialized in the KeyValue serialization format If the KeyValue (Cell's) serialization format changes this method cannot be used.
      Parameters:
      comparator - the CellComparator to use for comparison
      left - the cell to be compared
      key - the serialized key part of a KeyValue
      offset - the offset in the key byte[]
      length - the length of the key byte[]
      Returns:
      an int greater than 0 if left is greater than right lesser than 0 if left is lesser than right equal to 0 if left is equal to right
    • compareWithoutRow

      static final int compareWithoutRow(CellComparator comparator, ExtendedCell left, byte[] right, int roffset, int rlength, short rowlength)
      Compare columnFamily, qualifier, timestamp, and key type (everything except the row). This method is used both in the normal comparator and the "same-prefix" comparator. Note that we are assuming that row portions of both KVs have already been parsed and found identical, and we don't validate that assumption here.
      Parameters:
      comparator - the CellComparator to use for comparison
      left - the cell to be compared
      right - the serialized key part of a key-value
      roffset - the offset in the key byte[]
      rlength - the length of the key byte[]
      rowlength - the row length
      Returns:
      greater than 0 if left cell is bigger, less than 0 if right cell is bigger, 0 if both cells are equal
    • createNextOnRowCol

      Return a new cell is located following input cell. If both of type and timestamp are minimum, the input cell will be returned directly.
    • createNextOnRowCol

      static ExtendedCell createNextOnRowCol(ExtendedCell cell, long ts, byte type)
    • estimatedSerializedSizeOf

      public static int estimatedSerializedSizeOf(Cell cell)
      Estimate based on keyvalue's serialization format in the RPC layer. Note that there is an extra SIZEOF_INT added to the size here that indicates the actual length of the cell for cases where cell's are serialized in a contiguous format (For eg in RPCs).
      Returns:
      Estimate of the cell size in bytes plus an extra SIZEOF_INT indicating the actual cell length.
    • estimatedSerializedSizeOfKey

      public static int estimatedSerializedSizeOfKey(Cell cell)
      Calculates the serialized key size. We always serialize in the KeyValue's serialization format.
      Parameters:
      cell - the cell for which the key size has to be calculated.
      Returns:
      the key size
    • getCellKeySerializedAsKeyValueKey

      public static byte[] getCellKeySerializedAsKeyValueKey(ExtendedCell cell)
      This method exists just to encapsulate how we serialize keys. To be replaced by a factory that we query to figure what the Cell implementation is and then, what serialization engine to use and further, how to serialize the key for inclusion in hfile index. TODO.
      Returns:
      The key portion of the Cell serialized in the old-school KeyValue way or null if passed a null cell
    • createFirstOnRow

      public static ExtendedCell createFirstOnRow(Cell cell)
      Create a Cell that is smaller than all other possible Cells for the given Cell's row.
      Returns:
      First possible Cell on passed Cell's row.
    • createFirstOnRow

      public static ExtendedCell createFirstOnRow(byte[] row, int roffset, short rlength)
    • createFirstOnRow

      public static ExtendedCell createFirstOnRow(byte[] row, byte[] family, byte[] col)
    • createFirstOnRow

      public static ExtendedCell createFirstOnRow(byte[] row, int roffset, short rlength, byte[] family, int foffset, byte flength, byte[] col, int coffset, int clength)
    • createFirstOnRow

      public static ExtendedCell createFirstOnRow(byte[] row)
    • createFirstOnRowFamily

      public static ExtendedCell createFirstOnRowFamily(Cell cell, byte[] fArray, int foff, int flen)
    • createFirstOnRowCol

      public static ExtendedCell createFirstOnRowCol(Cell cell)
    • createFirstOnNextRow

      public static ExtendedCell createFirstOnNextRow(Cell cell)
    • createFirstOnRowCol

      public static ExtendedCell createFirstOnRowCol(Cell cell, byte[] qArray, int qoffest, int qlength)
      Create a Cell that is smaller than all other possible Cells for the given Cell's rk:cf and passed qualifier.
      Returns:
      Last possible Cell on passed Cell's rk:cf and passed qualifier.
    • createFirstOnRowColTS

      public static ExtendedCell createFirstOnRowColTS(Cell cell, long ts)
      Creates the first cell with the row/family/qualifier of this cell and the given timestamp. Uses the "maximum" type that guarantees that the new cell is the lowest possible for this combination of row, family, qualifier, and timestamp. This cell's own timestamp is ignored.
      Parameters:
      cell - - cell
    • createLastOnRow

      public static ExtendedCell createLastOnRow(Cell cell)
      Create a Cell that is larger than all other possible Cells for the given Cell's row.
      Returns:
      Last possible Cell on passed Cell's row.
    • createLastOnRow

      public static ExtendedCell createLastOnRow(byte[] row)
    • createLastOnRowCol

      public static ExtendedCell createLastOnRowCol(Cell cell)
      Create a Cell that is larger than all other possible Cells for the given Cell's rk:cf:q. Used in creating "fake keys" for the multi-column Bloom filter optimization to skip the row/column we already know is not in the file.
      Returns:
      Last possible Cell on passed Cell's rk:cf:q.
    • createFirstDeleteFamilyCellOnRow

      public static ExtendedCell createFirstDeleteFamilyCellOnRow(byte[] row, byte[] fam)
      Create a Delete Family Cell for the specified row and family that would be smaller than all other possible Delete Family KeyValues that have the same row and family. Used for seeking.
      Parameters:
      row - - row key (arbitrary byte array)
      fam - - family name
      Returns:
      First Delete Family possible key on passed row.
    • ensureExtendedCell

      public static ExtendedCell ensureExtendedCell(Cell cell)
      In fact, in HBase, all Cells are ExtendedCells. We do not expect users to implement their own Cell types, except some special projects like Phoenix, where they just use KeyValue and ExtendedCell directly.
      Returns:
      the original cell which has already been cast to an ExtendedCell.
      Throws:
      IllegalArgumentException - if the given cell is not an ExtendedCell.
    • equals

      public static boolean equals(ExtendedCell a, ExtendedCell b)
    • createExtendedCellScanner

      public static ExtendedCellScanner createExtendedCellScanner(List<? extends ExtendedCellScannable> cellScannerables)
      Returns ExtendedCellScanner interface over cellIterables
    • createExtendedCellScanner

      Flatten the map of cells out under the ExtendedCellScanner
      Parameters:
      map - Map of Cell Lists; for example, the map of families to ExtendedCells that is used inside Put, etc., keeping Cells organized by family.
      Returns:
      ExtendedCellScanner interface over cellIterable
    • createExtendedCellScanner

      Returns CellScanner interface over cellArray
    • createExtendedCellScanner

      Returns ExtendedCellScanner interface over cellIterable
    • createExtendedCellScanner

      Returns ExtendedCellScanner interface over cellIterable or null if cells is null
    • code2Type

      public static Cell.Type code2Type(byte code)
    • getTypeByte

      public static byte getTypeByte(Cell c)
    • getSequenceId

      public static long getSequenceId(Cell c)