Class CompositeBuffer

  • All Implemented Interfaces:
    Buffer

    public class CompositeBuffer
    extends Object
    implements Buffer
    An implementation of Buffer which contains multiple partial buffers for network data communication.
    • Constructor Detail

      • CompositeBuffer

        public CompositeBuffer​(Buffer.DataType dataType,
                               int length,
                               boolean isCompressed)
      • CompositeBuffer

        public CompositeBuffer​(BufferHeader header)
    • Method Detail

      • isBuffer

        public boolean isBuffer()
        Description copied from interface: Buffer
        Returns whether this buffer represents a buffer or an event.
        Specified by:
        isBuffer in interface Buffer
        Returns:
        true if this is a real buffer, false if this is an event
      • recycleBuffer

        public void recycleBuffer()
        Description copied from interface: Buffer
        Releases this buffer once, i.e. reduces the reference count and recycles the buffer if the reference count reaches 0.
        Specified by:
        recycleBuffer in interface Buffer
        See Also:
        Buffer.retainBuffer()
      • retainBuffer

        public Buffer retainBuffer()
        Description copied from interface: Buffer
        Retains this buffer for further use, increasing the reference counter by 1.
        Specified by:
        retainBuffer in interface Buffer
        Returns:
        this instance (for chained calls)
        See Also:
        Buffer.recycleBuffer()
      • getSize

        public int getSize()
        Description copied from interface: Buffer
        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.

        Specified by:
        getSize in interface Buffer
        Returns:
        writer index (from 0 (inclusive) to the size of the backing MemorySegment (inclusive))
      • setAllocator

        public void setAllocator​(org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator allocator)
        Description copied from interface: Buffer
        Sets the buffer allocator for use in netty.
        Specified by:
        setAllocator in interface Buffer
        Parameters:
        allocator - netty buffer allocator
      • asByteBuf

        public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf asByteBuf()
        Specified by:
        asByteBuf in interface Buffer
        Returns:
        self as ByteBuf implementation.
      • isCompressed

        public boolean isCompressed()
        Specified by:
        isCompressed in interface Buffer
        Returns:
        whether the buffer is compressed or not.
      • numPartialBuffers

        public int numPartialBuffers()
      • getFullBufferData

        public Buffer getFullBufferData​(MemorySegment segment)
        Returns the full buffer data in one piece of MemorySegment. If there is multiple partial buffers, the partial data will be copied to the given target MemorySegment.
      • addPartialBuffer

        public void addPartialBuffer​(Buffer buffer)
      • missingLength

        public int missingLength()
      • getMemorySegment

        public MemorySegment getMemorySegment()
        Description copied from interface: Buffer
        Returns the underlying memory segment. This method is dangerous since it ignores read only protections and omits slices. Use it only along the Buffer.getMemorySegmentOffset().

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

        Specified by:
        getMemorySegment in interface Buffer
        Returns:
        the memory segment backing this buffer
      • getMemorySegmentOffset

        public int getMemorySegmentOffset()
        Description copied from interface: Buffer
        This method will be removed in the future. For writing use BufferBuilder.
        Specified by:
        getMemorySegmentOffset in interface Buffer
        Returns:
        the offset where this (potential slice) Buffer's data start in the underlying memory segment.
      • getRecycler

        public BufferRecycler getRecycler()
        Description copied from interface: Buffer
        Gets the buffer's recycler.
        Specified by:
        getRecycler in interface Buffer
        Returns:
        buffer recycler
      • setRecycler

        public void setRecycler​(BufferRecycler bufferRecycler)
        Description copied from interface: Buffer
        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.

        Specified by:
        setRecycler in interface Buffer
        Parameters:
        bufferRecycler - the new buffer recycler
      • isRecycled

        public boolean isRecycled()
        Description copied from interface: Buffer
        Returns whether this buffer has been recycled or not.
        Specified by:
        isRecycled in interface Buffer
        Returns:
        true if already recycled, false otherwise
      • readOnlySlice

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

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

        Specified by:
        readOnlySlice in interface Buffer
        Returns:
        a read-only sliced buffer
      • readOnlySlice

        public Buffer readOnlySlice​(int index,
                                    int length)
        Description copied from interface: Buffer
        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.

        Specified by:
        readOnlySlice in interface Buffer
        Parameters:
        index - the index to start from
        length - the length of the slice
        Returns:
        a read-only sliced buffer
      • getMaxCapacity

        public int getMaxCapacity()
        Description copied from interface: Buffer
        Returns the maximum size of the buffer, i.e. the capacity of the underlying MemorySegment.
        Specified by:
        getMaxCapacity in interface Buffer
        Returns:
        size of the buffer
      • getReaderIndex

        public int getReaderIndex()
        Description copied from interface: Buffer
        Returns the reader index of this buffer.

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

        Specified by:
        getReaderIndex in interface Buffer
        Returns:
        reader index (from 0 (inclusive) to the size of the backing MemorySegment (inclusive))
      • setReaderIndex

        public void setReaderIndex​(int readerIndex)
        Description copied from interface: Buffer
        Sets the reader index of this buffer.
        Specified by:
        setReaderIndex in interface Buffer
      • setSize

        public void setSize​(int writerIndex)
        Description copied from interface: Buffer
        Sets the size of the written data, i.e. the writer index, of this buffer.
        Specified by:
        setSize in interface Buffer
      • getNioBuffer

        public ByteBuffer getNioBuffer​(int index,
                                       int length)
        Description copied from interface: Buffer
        Gets a new ByteBuffer instance wrapping this buffer's bytes.

        Please note that neither read nor write index are updated by the returned buffer.

        Specified by:
        getNioBuffer in interface Buffer
        Returns:
        byte buffer sharing the contents of the underlying memory segment
        See Also:
        Buffer.getNioBufferReadable()
      • setCompressed

        public void setCompressed​(boolean isCompressed)
        Description copied from interface: Buffer
        Tags the buffer as compressed or uncompressed.
        Specified by:
        setCompressed in interface Buffer
      • setDataType

        public void setDataType​(Buffer.DataType dataType)
        Description copied from interface: Buffer
        Sets the type of data this buffer represents.
        Specified by:
        setDataType in interface Buffer