Interface Buffer

  • All Known Implementing Classes:
    CompositeBuffer, FileRegionBuffer, NetworkBuffer, ReadOnlySlicedNetworkBuffer

    public interface Buffer
    Wrapper for pooled MemorySegment instances with reference counting.

    This is similar to Netty's ByteBuf with some extensions and restricted to the methods our use cases outside Netty handling use. In particular, we use two different indexes for read and write operations, i.e. the reader and writer index (size of written data), which specify three regions inside the memory segment:

         +-------------------+----------------+----------------+
         | discardable bytes | readable bytes | writable bytes |
         +-------------------+----------------+----------------+
         |                   |                |                |
         0      <=      readerIndex  <=  writerIndex   <=  max capacity
     

    Our non-Netty usages of this Buffer class either rely on the underlying getMemorySegment() directly, or on ByteBuffer wrappers of this buffer which do not modify either index, so the indices need to be updated manually via setReaderIndex(int) and setSize(int).

    • Method Detail

      • isBuffer

        boolean isBuffer()
        Returns whether this buffer represents a buffer or an event.
        Returns:
        true if this is a real buffer, false if this is an event
      • getMemorySegment

        @Deprecated
        MemorySegment getMemorySegment()
        Deprecated.
        Returns the underlying memory segment. This method is dangerous since it ignores read only protections and omits slices. Use it only along the getMemorySegmentOffset().

        This method will be removed in the future. For writing use BufferBuilder.

        Returns:
        the memory segment backing this buffer
      • getMemorySegmentOffset

        @Deprecated
        int getMemorySegmentOffset()
        Deprecated.
        This method will be removed in the future. For writing use BufferBuilder.
        Returns:
        the offset where this (potential slice) Buffer's data start in the underlying memory segment.
      • getRecycler

        BufferRecycler getRecycler()
        Gets the buffer's recycler.
        Returns:
        buffer recycler
      • setRecycler

        void setRecycler​(BufferRecycler bufferRecycler)
        Sets the buffer's recycler.

        Note that updating the recycler is an unsafe operation and this method cannot guarantee thread safety. It is important for the caller to fully understand the consequences of calling this method. Incorrectly updating the buffer recycler can result in a leak of the buffer due to using a wrong recycler to recycle buffer. Therefore, be careful when calling this method.

        Parameters:
        bufferRecycler - the new buffer recycler
      • recycleBuffer

        void recycleBuffer()
        Releases this buffer once, i.e. reduces the reference count and recycles the buffer if the reference count reaches 0.
        See Also:
        retainBuffer()
      • isRecycled

        boolean isRecycled()
        Returns whether this buffer has been recycled or not.
        Returns:
        true if already recycled, false otherwise
      • retainBuffer

        Buffer retainBuffer()
        Retains this buffer for further use, increasing the reference counter by 1.
        Returns:
        this instance (for chained calls)
        See Also:
        recycleBuffer()
      • readOnlySlice

        Buffer readOnlySlice()
        Returns a read-only slice of this buffer's readable bytes, i.e. between getReaderIndex() and getSize().

        Reader and writer indices as well as markers are not shared. Reference counters are shared but the slice is not retained automatically.

        Returns:
        a read-only sliced buffer
      • readOnlySlice

        Buffer readOnlySlice​(int index,
                             int length)
        Returns a read-only slice of this buffer.

        Reader and writer indices as well as markers are not shared. Reference counters are shared but the slice is not retained automatically.

        Parameters:
        index - the index to start from
        length - the length of the slice
        Returns:
        a read-only sliced buffer
      • getMaxCapacity

        int getMaxCapacity()
        Returns the maximum size of the buffer, i.e. the capacity of the underlying MemorySegment.
        Returns:
        size of the buffer
      • getReaderIndex

        int getReaderIndex()
        Returns the reader index of this buffer.

        This is where readable (unconsumed) bytes start in the backing memory segment.

        Returns:
        reader index (from 0 (inclusive) to the size of the backing MemorySegment (inclusive))
      • getSize

        int getSize()
        Returns the size of the written data, i.e. the writer index, of this buffer.

        This is where writable bytes start in the backing memory segment.

        Returns:
        writer index (from 0 (inclusive) to the size of the backing MemorySegment (inclusive))
      • getNioBufferReadable

        ByteBuffer getNioBufferReadable()
        Gets a new ByteBuffer instance wrapping this buffer's readable bytes, i.e. between getReaderIndex() and getSize().

        Please note that neither index is updated by the returned buffer.

        Returns:
        byte buffer sharing the contents of the underlying memory segment
      • setAllocator

        void setAllocator​(org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator allocator)
        Sets the buffer allocator for use in netty.
        Parameters:
        allocator - netty buffer allocator
      • asByteBuf

        org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf asByteBuf()
        Returns:
        self as ByteBuf implementation.
      • isCompressed

        boolean isCompressed()
        Returns:
        whether the buffer is compressed or not.
      • setCompressed

        void setCompressed​(boolean isCompressed)
        Tags the buffer as compressed or uncompressed.
      • getDataType

        Buffer.DataType getDataType()
        Gets the type of data this buffer represents.
      • setDataType

        void setDataType​(Buffer.DataType dataType)
        Sets the type of data this buffer represents.
      • toDebugString

        default String toDebugString​(boolean includeHash)