Class CompositeBuffer
- java.lang.Object
-
- org.apache.flink.runtime.io.network.buffer.CompositeBuffer
-
-
Nested Class Summary
-
Nested classes/interfaces inherited from interface org.apache.flink.runtime.io.network.buffer.Buffer
Buffer.DataType
-
-
Constructor Summary
Constructors Constructor Description CompositeBuffer(Buffer.DataType dataType, int length, boolean isCompressed)
CompositeBuffer(BufferHeader header)
-
Method Summary
All Methods Instance Methods Concrete Methods Modifier and Type Method Description void
addPartialBuffer(Buffer buffer)
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
asByteBuf()
Buffer.DataType
getDataType()
Gets the type of data this buffer represents.Buffer
getFullBufferData(MemorySegment segment)
Returns the full buffer data in one piece ofMemorySegment
.int
getMaxCapacity()
Returns the maximum size of the buffer, i.e. the capacity of the underlyingMemorySegment
.MemorySegment
getMemorySegment()
Returns the underlying memory segment.int
getMemorySegmentOffset()
This method will be removed in the future.ByteBuffer
getNioBuffer(int index, int length)
Gets a newByteBuffer
instance wrapping this buffer's bytes.ByteBuffer
getNioBufferReadable()
Gets a newByteBuffer
instance wrapping this buffer's readable bytes, i.e. betweenBuffer.getReaderIndex()
andBuffer.getSize()
.int
getReaderIndex()
Returns the reader index of this buffer.BufferRecycler
getRecycler()
Gets the buffer's recycler.int
getSize()
Returns the size of the written data, i.e. the writer index, of this buffer.boolean
isBuffer()
Returns whether this buffer represents a buffer or an event.boolean
isCompressed()
boolean
isRecycled()
Returns whether this buffer has been recycled or not.int
missingLength()
int
numPartialBuffers()
int
readableBytes()
Returns the number of readable bytes (same asBuffer.getSize()
-Buffer.getReaderIndex()
).Buffer
readOnlySlice()
Returns a read-only slice of this buffer's readable bytes, i.e. betweenBuffer.getReaderIndex()
andBuffer.getSize()
.Buffer
readOnlySlice(int index, int length)
Returns a read-only slice of this buffer.void
recycleBuffer()
Releases this buffer once, i.e. reduces the reference count and recycles the buffer if the reference count reaches 0.int
refCnt()
The current reference counter.Buffer
retainBuffer()
Retains this buffer for further use, increasing the reference counter by 1.void
setAllocator(org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator allocator)
Sets the buffer allocator for use in netty.void
setCompressed(boolean isCompressed)
Tags the buffer as compressed or uncompressed.void
setDataType(Buffer.DataType dataType)
Sets the type of data this buffer represents.void
setReaderIndex(int readerIndex)
Sets the reader index of this buffer.void
setRecycler(BufferRecycler bufferRecycler)
Sets the buffer's recycler.void
setSize(int writerIndex)
Sets the size of the written data, i.e. the writer index, of this buffer.-
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
-
Methods inherited from interface org.apache.flink.runtime.io.network.buffer.Buffer
toDebugString
-
-
-
-
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.
-
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 interfaceBuffer
- 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 interfaceBuffer
- 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 interfaceBuffer
- Returns:
- writer index (from 0 (inclusive) to the size of the backing
MemorySegment
(inclusive))
-
readableBytes
public int readableBytes()
Description copied from interface:Buffer
Returns the number of readable bytes (same asBuffer.getSize()
-Buffer.getReaderIndex()
).- Specified by:
readableBytes
in interfaceBuffer
-
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 interfaceBuffer
- Parameters:
allocator
- netty buffer allocator
-
asByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf asByteBuf()
-
isCompressed
public boolean isCompressed()
- Specified by:
isCompressed
in interfaceBuffer
- Returns:
- whether the buffer is compressed or not.
-
getDataType
public Buffer.DataType getDataType()
Description copied from interface:Buffer
Gets the type of data this buffer represents.- Specified by:
getDataType
in interfaceBuffer
-
numPartialBuffers
public int numPartialBuffers()
-
getFullBufferData
public Buffer getFullBufferData(MemorySegment segment)
Returns the full buffer data in one piece ofMemorySegment
. If there is multiple partial buffers, the partial data will be copied to the given targetMemorySegment
.
-
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 theBuffer.getMemorySegmentOffset()
.This method will be removed in the future. For writing use
BufferBuilder
.- Specified by:
getMemorySegment
in interfaceBuffer
- 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 useBufferBuilder
.- Specified by:
getMemorySegmentOffset
in interfaceBuffer
- 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 interfaceBuffer
- 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 interfaceBuffer
- 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 interfaceBuffer
- 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. betweenBuffer.getReaderIndex()
andBuffer.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 interfaceBuffer
- 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 interfaceBuffer
- Parameters:
index
- the index to start fromlength
- 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 underlyingMemorySegment
.- Specified by:
getMaxCapacity
in interfaceBuffer
- 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 interfaceBuffer
- 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 interfaceBuffer
-
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.
-
getNioBufferReadable
public ByteBuffer getNioBufferReadable()
Description copied from interface:Buffer
Gets a newByteBuffer
instance wrapping this buffer's readable bytes, i.e. betweenBuffer.getReaderIndex()
andBuffer.getSize()
.Please note that neither index is updated by the returned buffer.
- Specified by:
getNioBufferReadable
in interfaceBuffer
- Returns:
- byte buffer sharing the contents of the underlying memory segment
-
getNioBuffer
public ByteBuffer getNioBuffer(int index, int length)
Description copied from interface:Buffer
Gets a newByteBuffer
instance wrapping this buffer's bytes.Please note that neither read nor write index are updated by the returned buffer.
- Specified by:
getNioBuffer
in interfaceBuffer
- 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 interfaceBuffer
-
setDataType
public void setDataType(Buffer.DataType dataType)
Description copied from interface:Buffer
Sets the type of data this buffer represents.- Specified by:
setDataType
in interfaceBuffer
-
refCnt
public int refCnt()
Description copied from interface:Buffer
The current reference counter. Increased byBuffer.retainBuffer()
and decreased withBuffer.recycleBuffer()
.
-
-