Class ByteBuff

java.lang.Object
org.apache.hadoop.hbase.nio.ByteBuff
All Implemented Interfaces:
HBaseReferenceCounted, org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
Direct Known Subclasses:
MultiByteBuff, SingleByteBuff

@Private public abstract class ByteBuff extends Object implements HBaseReferenceCounted
An abstract class that abstracts out as to how the byte buffers are used, either single or multiple. We have this interface because the java's ByteBuffers cannot be sub-classed. This class provides APIs similar to the ones provided in java's nio ByteBuffers and allows you to do positional reads/writes and relative reads and writes on the underlying BB. In addition to it, we have some additional APIs which helps us in the read path.
The ByteBuff implement HBaseReferenceCounted interface which mean need to maintains a RefCnt inside, if ensure that the ByteBuff won't be used any more, we must do a release() to recycle its NIO ByteBuffers. when considering the duplicate() or slice(), releasing either the duplicated one or the original one will free its memory, because they share the same NIO ByteBuffers. when you want to retain the NIO ByteBuffers even if the origin one called release(), you can do like this:
   ByteBuff original = ...;
   ByteBuff dup = original.duplicate();
   dup.retain();
   original.release();
   // The NIO buffers can still be accessed unless you release the duplicated one
   dup.get(...);
   dup.release();
   // Both the original and dup can not access the NIO buffers any more.
 
  • Nested Class Summary

    Nested Classes
    Modifier and Type
    Class
    Description
    (package private) static interface 
    Functional interface for Channel read
  • Field Summary

    Fields
    Modifier and Type
    Field
    Description
    (package private) static final ByteBuff.ChannelReader
     
    (package private) static final ByteBuff.ChannelReader
     
    private static final int
     
    protected RefCnt
     
    private static final String
     
  • Constructor Summary

    Constructors
    Constructor
    Description
     
  • Method Summary

    Modifier and Type
    Method
    Description
    abstract byte[]
    Returns the byte[] if the underlying BB has single BB and hasArray true
    abstract int
    Returns the arrayOffset of the byte[] incase of a single BB backed ByteBuff
    abstract ByteBuffer
    asSubByteBuffer(int length)
    Returns bytes from current position till length specified, as a single ByteBuffer.
    abstract void
    asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair)
    Returns bytes from given offset till length specified, as a single ByteBuffer.
    abstract int
    Returns the total capacity of this ByteBuff.
    protected void
    Checks that there are still references to the buffer.
    static int
    compareTo(ByteBuff buf1, int o1, int len1, ByteBuff buf2, int o2, int len2)
    Compares two ByteBuffs
    abstract ByteBuff
    Returns an ByteBuff which is a duplicate version of this ByteBuff.
    abstract byte
    get()
    A relative method that returns byte at the current position.
    abstract void
    get(byte[] dst)
    Copies the content from this ByteBuff's current position to the byte array and fills it.
    abstract void
    get(byte[] dst, int offset, int length)
    Copies the specified number of bytes from this ByteBuff's current position to the byte[]'s offset.
    abstract byte
    get(int index)
    Fetches the byte at the given index.
    abstract void
    get(int sourceOffset, byte[] dst, int offset, int length)
    Copies the specified number of bytes from this ByteBuff's given position to the byte[]'s offset.
    abstract void
    get(ByteBuffer out, int sourceOffset, int length)
    Copies the content from this ByteBuff to a ByteBuffer Note : This will advance the position marker of out but not change the position maker for this ByteBuff
    abstract byte
    Fetches the byte at the given offset from current position.
    abstract int
    Returns the int value at the current position.
    abstract int
    getInt(int index)
    Fetches the int at the given index.
    abstract int
    getIntAfterPosition(int offset)
    Fetches the int value at the given offset from current position.
    abstract long
    Returns the long value at the current position.
    abstract long
    getLong(int index)
    Fetches the long at the given index.
    abstract long
    Fetches the long value at the given offset from current position.
     
    abstract short
    Returns the short value at the current position.
    abstract short
    getShort(int index)
    Fetches the short value at the given index.
    abstract short
    Fetches the short value at the given offset from current position.
    abstract boolean
    Returns true or false if the underlying BB support hasArray
    abstract boolean
    Returns true if there are elements between the current position and the limit.
    abstract int
    Returns the limit of this ByteBuff
    abstract ByteBuff
    limit(int limit)
    Marks the limit of this ByteBuff
    abstract ByteBuff
    Marks the current position of the ByteBuff
    abstract ByteBuff
    moveBack(int len)
    Jumps back the current position of this ByteBuff by specified length.
    abstract ByteBuffer[]
     
    abstract int
    Returns this ByteBuff's current position
    abstract ByteBuff
    position(int position)
    Sets this ByteBuff's position to the given value.
    abstract ByteBuff
    put(byte b)
    Writes a byte to this ByteBuff at the current position and increments the position
    abstract ByteBuff
    put(byte[] src)
    Copies from the given byte[] to this ByteBuff
    abstract ByteBuff
    put(byte[] src, int offset, int length)
    Copies from the given byte[] to this ByteBuff
    abstract ByteBuff
    put(int index, byte b)
    Writes a byte to this ByteBuff at the given index
    abstract ByteBuff
    put(int offset, ByteBuff src, int srcOffset, int length)
    Copies the contents from the src ByteBuff to this ByteBuff.
    abstract ByteBuff
    putInt(int value)
    Writes an int to this ByteBuff at its current position.
    abstract ByteBuff
    putLong(long value)
    Writes a long to this ByteBuff at its current position.
    abstract int
    read(FileChannel channel, long offset)
    Reads bytes from FileChannel into this ByteBuff
    abstract int
    Reads bytes from the given channel into this ByteBuf.
    static int
    read(ReadableByteChannel channel, ByteBuffer buf, long offset, ByteBuff.ChannelReader reader)
     
    static int
    Read integer from ByteBuff coded in 7 bits and increment position.
    static long
    readLong(ByteBuff in, int fitInBytes)
    Read long which was written to fitInBytes bytes and increment position.
    int
     
    boolean
     
    abstract int
    Returns the number of elements between the current position and the limit.
    abstract ByteBuff
    Similar to ByteBuffer.reset(), ensures that this ByteBuff is reset back to last marked position.
    abstract ByteBuff
    Rewinds this ByteBuff and the position is set to 0
    abstract ByteBuff
    skip(int len)
    Jumps the current position of this ByteBuff by specified length.
    abstract ByteBuff
    Returns an ByteBuff which is a sliced version of this ByteBuff.
    byte[]
    Copy the content from this ByteBuff to a byte[].
    abstract byte[]
    toBytes(int offset, int length)
    Copy the content from this ByteBuff to a byte[] based on the given offset and length.
     
    Calling this method in strategic locations where ByteBuffs are referenced may help diagnose potential buffer leaks.
    touch(Object hint)
     
    static ByteBuff
    wrap(ByteBuffer buffer)
     
    static ByteBuff
    wrap(ByteBuffer[] buffers)
     
    static ByteBuff
     
    static ByteBuff
    wrap(ByteBuffer[] buffers, RefCnt refCnt)
    In theory, the upstream should never construct an ByteBuff by passing an given refCnt, so please don't use this public method in other place.
    private static ByteBuff
    wrap(ByteBuffer buffer, RefCnt refCnt)
     
    static ByteBuff
    wrap(List<ByteBuffer> buffers)
     
    static ByteBuff
     
    private static ByteBuff
    wrap(List<ByteBuffer> buffers, RefCnt refCnt)
     
    abstract int
    write(FileChannel channel, long offset)
    Write this ByteBuff's data into target file

    Methods inherited from class java.lang.Object

    clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait

    Methods inherited from interface org.apache.hadoop.hbase.nio.HBaseReferenceCounted

    release, retain

    Methods inherited from interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted

    retain
  • Field Details

  • Constructor Details

  • Method Details

    • checkRefCount

      protected void checkRefCount()
      Checks that there are still references to the buffer. This protects against the case where a ByteBuff method (i.e. slice, get, etc) could be called against a buffer whose backing data may have been released. We only need to do this check if the refCnt has a recycler. If there's no recycler, the backing data will be handled by normal java GC and won't get incorrectly released. So we can avoid the overhead of checking the refCnt on every call. See HBASE-27710.
    • refCnt

      public int refCnt()
      Specified by:
      refCnt in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
    • release

      public boolean release()
      Specified by:
      release in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
    • position

      public abstract int position()
      Returns this ByteBuff's current position
    • position

      public abstract ByteBuff position(int position)
      Sets this ByteBuff's position to the given value.
      Returns:
      this object
    • skip

      public abstract ByteBuff skip(int len)
      Jumps the current position of this ByteBuff by specified length.
      Parameters:
      len - the length to be skipped
    • moveBack

      public abstract ByteBuff moveBack(int len)
      Jumps back the current position of this ByteBuff by specified length.
      Parameters:
      len - the length to move back
    • capacity

      public abstract int capacity()
      Returns the total capacity of this ByteBuff.
    • limit

      public abstract int limit()
      Returns the limit of this ByteBuff
    • limit

      public abstract ByteBuff limit(int limit)
      Marks the limit of this ByteBuff
    • rewind

      public abstract ByteBuff rewind()
      Rewinds this ByteBuff and the position is set to 0
    • mark

      public abstract ByteBuff mark()
      Marks the current position of the ByteBuff
    • asSubByteBuffer

      public abstract ByteBuffer asSubByteBuffer(int length)
      Returns bytes from current position till length specified, as a single ByteBuffer. When all these bytes happen to be in a single ByteBuffer, which this object wraps, that ByteBuffer item as such will be returned. So users are warned not to change the position or limit of this returned ByteBuffer. The position of the returned byte buffer is at the begin of the required bytes. When the required bytes happen to span across multiple ByteBuffers, this API will copy the bytes to a newly created ByteBuffer of required size and return that.
      Parameters:
      length - number of bytes required.
      Returns:
      bytes from current position till length specified, as a single ByteButter.
    • asSubByteBuffer

      public abstract void asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair)
      Returns bytes from given offset till length specified, as a single ByteBuffer. When all these bytes happen to be in a single ByteBuffer, which this object wraps, that ByteBuffer item as such will be returned (with offset in this ByteBuffer where the bytes starts). So users are warned not to change the position or limit of this returned ByteBuffer. When the required bytes happen to span across multiple ByteBuffers, this API will copy the bytes to a newly created ByteBuffer of required size and return that.
      Parameters:
      offset - the offset in this ByteBuff from where the subBuffer should be created
      length - the length of the subBuffer
      pair - a pair that will have the bytes from the current position till length specified, as a single ByteBuffer and offset in that Buffer where the bytes starts. Since this API gets called in a loop we are passing a pair to it which could be created outside the loop and the method would set the values on the pair that is passed in by the caller. Thus it avoids more object creations that would happen if the pair that is returned is created by this method every time.
    • remaining

      public abstract int remaining()
      Returns the number of elements between the current position and the limit.
    • hasRemaining

      public abstract boolean hasRemaining()
      Returns true if there are elements between the current position and the limit.
    • reset

      public abstract ByteBuff reset()
      Similar to ByteBuffer.reset(), ensures that this ByteBuff is reset back to last marked position.
      Returns:
      This ByteBuff
    • slice

      public abstract ByteBuff slice()
      Returns an ByteBuff which is a sliced version of this ByteBuff. The position, limit and mark of the new ByteBuff will be independent than that of the original ByteBuff. The content of the new ByteBuff will start at this ByteBuff's current position
      Returns:
      a sliced ByteBuff
    • duplicate

      public abstract ByteBuff duplicate()
      Returns an ByteBuff which is a duplicate version of this ByteBuff. The position, limit and mark of the new ByteBuff will be independent than that of the original ByteBuff. The content of the new ByteBuff will start at this ByteBuff's current position The position, limit and mark of the new ByteBuff would be identical to this ByteBuff in terms of values.
      Returns:
      a sliced ByteBuff
    • get

      public abstract byte get()
      A relative method that returns byte at the current position. Increments the current position by the size of a byte.
      Returns:
      the byte at the current position
    • get

      public abstract byte get(int index)
      Fetches the byte at the given index. Does not change position of the underlying ByteBuffers
      Returns:
      the byte at the given index
    • getByteAfterPosition

      public abstract byte getByteAfterPosition(int offset)
      Fetches the byte at the given offset from current position. Does not change position of the underlying ByteBuffers.
      Returns:
      the byte value at the given index.
    • put

      public abstract ByteBuff put(byte b)
      Writes a byte to this ByteBuff at the current position and increments the position
      Returns:
      this object
    • put

      public abstract ByteBuff put(int index, byte b)
      Writes a byte to this ByteBuff at the given index
      Returns:
      this object
    • get

      public abstract void get(byte[] dst, int offset, int length)
      Copies the specified number of bytes from this ByteBuff's current position to the byte[]'s offset. Also advances the position of the ByteBuff by the given length.
      Parameters:
      dst - the byte[] to which the ByteBuff's content is to be copied
      offset - within the current array
      length - upto which the bytes to be copied
    • get

      public abstract void get(int sourceOffset, byte[] dst, int offset, int length)
      Copies the specified number of bytes from this ByteBuff's given position to the byte[]'s offset. The position of the ByteBuff remains in the current position only
      Parameters:
      sourceOffset - the offset in this ByteBuff from where the copy should happen
      dst - the byte[] to which the ByteBuff's content is to be copied
      offset - within the current array
      length - upto which the bytes to be copied
    • get

      public abstract void get(byte[] dst)
      Copies the content from this ByteBuff's current position to the byte array and fills it. Also advances the position of the ByteBuff by the length of the byte[].
      Parameters:
      dst - the byte[] to which the ByteBuff's content is to be copied
    • put

      public abstract ByteBuff put(byte[] src, int offset, int length)
      Copies from the given byte[] to this ByteBuff
      Parameters:
      src - source byte array
      offset - the position in the byte array from which the copy should be done
      length - the length upto which the copy should happen
      Returns:
      this ByteBuff
    • put

      public abstract ByteBuff put(byte[] src)
      Copies from the given byte[] to this ByteBuff
      Parameters:
      src - source byte array
      Returns:
      this ByteBuff
    • hasArray

      public abstract boolean hasArray()
      Returns true or false if the underlying BB support hasArray
    • array

      public abstract byte[] array()
      Returns the byte[] if the underlying BB has single BB and hasArray true
    • arrayOffset

      public abstract int arrayOffset()
      Returns the arrayOffset of the byte[] incase of a single BB backed ByteBuff
    • getShort

      public abstract short getShort()
      Returns the short value at the current position. Also advances the position by the size of short.
    • getShort

      public abstract short getShort(int index)
      Fetches the short value at the given index. Does not change position of the underlying ByteBuffers. The caller is sure that the index will be after the current position of this ByteBuff. So even if the current short does not fit in the current item we can safely move to the next item and fetch the remaining bytes forming the short
      Returns:
      the short value at the given index
    • getShortAfterPosition

      public abstract short getShortAfterPosition(int offset)
      Fetches the short value at the given offset from current position. Does not change position of the underlying ByteBuffers.
      Returns:
      the short value at the given index.
    • getInt

      public abstract int getInt()
      Returns the int value at the current position. Also advances the position by the size of int.
    • putInt

      public abstract ByteBuff putInt(int value)
      Writes an int to this ByteBuff at its current position. Also advances the position by size of int.
    • getInt

      public abstract int getInt(int index)
      Fetches the int at the given index. Does not change position of the underlying ByteBuffers. Even if the current int does not fit in the current item we can safely move to the next item and fetch the remaining bytes forming the int.
    • getIntAfterPosition

      public abstract int getIntAfterPosition(int offset)
      Fetches the int value at the given offset from current position. Does not change position of the underlying ByteBuffers.
    • getLong

      public abstract long getLong()
      Returns the long value at the current position. Also advances the position by the size of long.
    • putLong

      public abstract ByteBuff putLong(long value)
      Writes a long to this ByteBuff at its current position. Also advances the position by size of long.
    • getLong

      public abstract long getLong(int index)
      Fetches the long at the given index. Does not change position of the underlying ByteBuffers. The caller is sure that the index will be after the current position of this ByteBuff. So even if the current long does not fit in the current item we can safely move to the next item and fetch the remaining bytes forming the long
      Returns:
      the long value at the given index
    • getLongAfterPosition

      public abstract long getLongAfterPosition(int offset)
      Fetches the long value at the given offset from current position. Does not change position of the underlying ByteBuffers.
      Returns:
      the long value at the given index.
    • toBytes

      public byte[] toBytes()
      Copy the content from this ByteBuff to a byte[].
    • toBytes

      public abstract byte[] toBytes(int offset, int length)
      Copy the content from this ByteBuff to a byte[] based on the given offset and length.
    • get

      public abstract void get(ByteBuffer out, int sourceOffset, int length)
      Copies the content from this ByteBuff to a ByteBuffer Note : This will advance the position marker of out but not change the position maker for this ByteBuff
      Parameters:
      out - the ByteBuffer to which the copy has to happen
      sourceOffset - the offset in the ByteBuff from which the elements has to be copied
      length - the length in this ByteBuff upto which the elements has to be copied
    • put

      public abstract ByteBuff put(int offset, ByteBuff src, int srcOffset, int length)
      Copies the contents from the src ByteBuff to this ByteBuff. This will be absolute positional copying and won't affect the position of any of the buffers.
      Parameters:
      offset - the position in this ByteBuff to which the copy should happen
      src - the src ByteBuff
      srcOffset - the offset in the src ByteBuff from where the elements should be read
      length - the length up to which the copy should happen
    • read

      public abstract int read(ReadableByteChannel channel) throws IOException
      Reads bytes from the given channel into this ByteBuf.
      Throws:
      IOException
    • read

      public abstract int read(FileChannel channel, long offset) throws IOException
      Reads bytes from FileChannel into this ByteBuff
      Throws:
      IOException
    • write

      public abstract int write(FileChannel channel, long offset) throws IOException
      Write this ByteBuff's data into target file
      Throws:
      IOException
    • read

      public static int read(ReadableByteChannel channel, ByteBuffer buf, long offset, ByteBuff.ChannelReader reader) throws IOException
      Throws:
      IOException
    • readCompressedInt

      public static int readCompressedInt(ByteBuff buf)
      Read integer from ByteBuff coded in 7 bits and increment position.
    • compareTo

      public static int compareTo(ByteBuff buf1, int o1, int len1, ByteBuff buf2, int o2, int len2)
      Compares two ByteBuffs
      Parameters:
      buf1 - the first ByteBuff
      o1 - the offset in the first ByteBuff from where the compare has to happen
      len1 - the length in the first ByteBuff upto which the compare has to happen
      buf2 - the second ByteBuff
      o2 - the offset in the second ByteBuff from where the compare has to happen
      len2 - the length in the second ByteBuff upto which the compare has to happen
      Returns:
      Positive if buf1 is bigger than buf2, 0 if they are equal, and negative if buf1 is smaller than buf2.
    • readLong

      public static long readLong(ByteBuff in, int fitInBytes)
      Read long which was written to fitInBytes bytes and increment position.
      Parameters:
      fitInBytes - In how many bytes given long is stored.
      Returns:
      The value of parsed long.
    • nioByteBuffers

      public abstract ByteBuffer[] nioByteBuffers()
    • toString

      public String toString()
      Overrides:
      toString in class Object
    • wrap

      public static ByteBuff wrap(ByteBuffer[] buffers, RefCnt refCnt)
      In theory, the upstream should never construct an ByteBuff by passing an given refCnt, so please don't use this public method in other place. Make the method public here because the BucketEntry#wrapAsCacheable in hbase-server module will use its own refCnt and ByteBuffers from IOEngine to composite an HFileBlock's ByteBuff, we didn't find a better way so keep the public way here.
    • wrap

      public static ByteBuff wrap(ByteBuffer[] buffers, ByteBuffAllocator.Recycler recycler)
    • wrap

      public static ByteBuff wrap(ByteBuffer[] buffers)
    • wrap

      public static ByteBuff wrap(List<ByteBuffer> buffers, ByteBuffAllocator.Recycler recycler)
    • wrap

      public static ByteBuff wrap(List<ByteBuffer> buffers)
    • wrap

      public static ByteBuff wrap(ByteBuffer buffer)
    • touch

      public ByteBuff touch()
      Calling this method in strategic locations where ByteBuffs are referenced may help diagnose potential buffer leaks. We pass the buffer itself as a default hint, but one can use touch(Object) to pass their own hint as well.
      Specified by:
      touch in interface HBaseReferenceCounted
      Specified by:
      touch in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
    • touch

      public ByteBuff touch(Object hint)
      Specified by:
      touch in interface HBaseReferenceCounted
      Specified by:
      touch in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
    • getRefCnt

      public RefCnt getRefCnt()
    • wrap

      private static ByteBuff wrap(List<ByteBuffer> buffers, RefCnt refCnt)
    • wrap

      private static ByteBuff wrap(ByteBuffer buffer, RefCnt refCnt)