public interface Buffer
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)
.
Modifier and Type | Interface and Description |
---|---|
static class |
Buffer.DataType
Used to identify the type of data contained in the
Buffer so that we can get the
information without deserializing the serialized data. |
Modifier and Type | Method and 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.
|
MemorySegment |
getMemorySegment()
Deprecated.
|
int |
getMemorySegmentOffset()
Deprecated.
|
ByteBuffer |
getNioBuffer(int index,
int length)
Gets a new
ByteBuffer instance wrapping this buffer's bytes. |
ByteBuffer |
getNioBufferReadable()
Gets a new
ByteBuffer instance wrapping this buffer's readable bytes, i.e. |
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.
|
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 as
getSize() - getReaderIndex() ). |
Buffer |
readOnlySlice()
Returns a read-only slice of this buffer's readable bytes, i.e.
|
Buffer |
readOnlySlice(int index,
int length)
Returns a read-only slice of this buffer.
|
void |
recycleBuffer()
Releases this buffer once, i.e.
|
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 |
setSize(int writerIndex)
Sets the size of the written data, i.e.
|
default String |
toDebugString(boolean includeHash) |
boolean isBuffer()
@Deprecated MemorySegment getMemorySegment()
getMemorySegmentOffset()
.
This method will be removed in the future. For writing use BufferBuilder
.
@Deprecated int getMemorySegmentOffset()
BufferBuilder
.Buffer
's data start in the underlying
memory segment.BufferRecycler getRecycler()
void recycleBuffer()
retainBuffer()
boolean isRecycled()
Buffer retainBuffer()
recycleBuffer()
Buffer readOnlySlice()
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.
Buffer readOnlySlice(int index, int length)
Reader and writer indices as well as markers are not shared. Reference counters are shared
but the slice is not retained
automatically.
index
- the index to start fromlength
- the length of the sliceint getMaxCapacity()
MemorySegment
.int getReaderIndex()
This is where readable (unconsumed) bytes start in the backing memory segment.
MemorySegment
(inclusive))void setReaderIndex(int readerIndex) throws IndexOutOfBoundsException
IndexOutOfBoundsException
- if the index is less than 0 or greater than getSize()
int getSize()
This is where writable bytes start in the backing memory segment.
MemorySegment
(inclusive))void setSize(int writerIndex)
IndexOutOfBoundsException
- if the index is less than getReaderIndex()
or
greater than getMaxCapacity()
int readableBytes()
getSize()
- getReaderIndex()
).ByteBuffer getNioBufferReadable()
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.
ByteBuffer getNioBuffer(int index, int length) throws IndexOutOfBoundsException
ByteBuffer
instance wrapping this buffer's bytes.
Please note that neither read nor write index are updated by the returned buffer.
IndexOutOfBoundsException
- if the indexes are not without the buffer's boundsgetNioBufferReadable()
void setAllocator(org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator allocator)
allocator
- netty buffer allocatororg.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf asByteBuf()
boolean isCompressed()
void setCompressed(boolean isCompressed)
Buffer.DataType getDataType()
void setDataType(Buffer.DataType dataType)
int refCnt()
retainBuffer()
and decreased with recycleBuffer()
.default String toDebugString(boolean includeHash)
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.