Interface Buffer
-
- All Known Implementing Classes:
CompositeBuffer
,FileRegionBuffer
,NetworkBuffer
,ReadOnlySlicedNetworkBuffer
public interface Buffer
Wrapper for pooledMemorySegment
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 onByteBuffer
wrappers of this buffer which do not modify either index, so the indices need to be updated manually viasetReaderIndex(int)
andsetSize(int)
.
-
-
Nested Class Summary
Nested Classes Modifier and Type Interface Description static class
Buffer.DataType
Used to identify the type of data contained in theBuffer
so that we can get the information without deserializing the serialized data.
-
Method Summary
All Methods Instance Methods Abstract Methods Default Methods Deprecated Methods Modifier and Type Method Description org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
asByteBuf()
Buffer.DataType
getDataType()
Gets the type of data this buffer represents.int
getMaxCapacity()
Returns the maximum size of the buffer, i.e. the capacity of the underlyingMemorySegment
.MemorySegment
getMemorySegment()
Deprecated.int
getMemorySegmentOffset()
Deprecated.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. betweengetReaderIndex()
andgetSize()
.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
readableBytes()
Returns the number of readable bytes (same asgetSize()
-getReaderIndex()
).Buffer
readOnlySlice()
Returns a read-only slice of this buffer's readable bytes, i.e. betweengetReaderIndex()
andgetSize()
.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.default String
toDebugString(boolean includeHash)
-
-
-
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 thegetMemorySegmentOffset()
.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 useBufferBuilder
.- 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. betweengetReaderIndex()
andgetSize()
.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 fromlength
- 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 underlyingMemorySegment
.- 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))
-
setReaderIndex
void setReaderIndex(int readerIndex) throws IndexOutOfBoundsException
Sets the reader index of this buffer.- Throws:
IndexOutOfBoundsException
- if the index is less than 0 or greater thangetSize()
-
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))
-
setSize
void setSize(int writerIndex)
Sets the size of the written data, i.e. the writer index, of this buffer.- Throws:
IndexOutOfBoundsException
- if the index is less thangetReaderIndex()
or greater thangetMaxCapacity()
-
readableBytes
int readableBytes()
Returns the number of readable bytes (same asgetSize()
-getReaderIndex()
).
-
getNioBufferReadable
ByteBuffer getNioBufferReadable()
Gets a newByteBuffer
instance wrapping this buffer's readable bytes, i.e. betweengetReaderIndex()
andgetSize()
.Please note that neither index is updated by the returned buffer.
- Returns:
- byte buffer sharing the contents of the underlying memory segment
-
getNioBuffer
ByteBuffer getNioBuffer(int index, int length) throws IndexOutOfBoundsException
Gets a newByteBuffer
instance wrapping this buffer's bytes.Please note that neither read nor write index are updated by the returned buffer.
- Returns:
- byte buffer sharing the contents of the underlying memory segment
- Throws:
IndexOutOfBoundsException
- if the indexes are not without the buffer's bounds- See Also:
getNioBufferReadable()
-
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.
-
refCnt
int refCnt()
The current reference counter. Increased byretainBuffer()
and decreased withrecycleBuffer()
.
-
toDebugString
default String toDebugString(boolean includeHash)
-
-