public class NetworkBuffer extends org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf implements Buffer
MemorySegment
instances.
NOTE: before using this buffer in the netty stack, a buffer allocator must be
set via setAllocator(ByteBufAllocator)
!
Buffer.DataType
Constructor and Description |
---|
NetworkBuffer(MemorySegment memorySegment,
BufferRecycler recycler)
Creates a new buffer instance backed by the given memorySegment with 0 for
the readerIndex and writerIndex.
|
NetworkBuffer(MemorySegment memorySegment,
BufferRecycler recycler,
Buffer.DataType dataType)
Creates a new buffer instance backed by the given memorySegment with 0 for
the readerIndex and writerIndex.
|
NetworkBuffer(MemorySegment memorySegment,
BufferRecycler recycler,
Buffer.DataType dataType,
boolean isCompressed,
int size)
Creates a new buffer instance backed by the given memorySegment with 0 for
the readerIndex and size as writerIndex.
|
NetworkBuffer(MemorySegment memorySegment,
BufferRecycler recycler,
Buffer.DataType dataType,
int size)
Creates a new buffer instance backed by the given memorySegment with 0 for
the readerIndex and size as writerIndex.
|
Modifier and Type | Method and Description |
---|---|
protected byte |
_getByte(int index) |
protected int |
_getInt(int index) |
protected int |
_getIntLE(int index) |
protected long |
_getLong(int index) |
protected long |
_getLongLE(int index) |
protected short |
_getShort(int index) |
protected short |
_getShortLE(int index) |
protected int |
_getUnsignedMedium(int index) |
protected int |
_getUnsignedMediumLE(int index) |
protected void |
_setByte(int index,
int value) |
protected void |
_setInt(int index,
int value) |
protected void |
_setIntLE(int index,
int value) |
protected void |
_setLong(int index,
long value) |
protected void |
_setLongLE(int index,
long value) |
protected void |
_setMedium(int index,
int value) |
protected void |
_setMediumLE(int index,
int value) |
protected void |
_setShort(int index,
int value) |
protected void |
_setShortLE(int index,
int value) |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator |
alloc() |
byte[] |
array() |
int |
arrayOffset() |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
asByteBuf() |
int |
capacity() |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
capacity(int newCapacity) |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
copy(int index,
int length) |
protected void |
deallocate() |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
getBytes(int index,
byte[] dst,
int dstIndex,
int length) |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
getBytes(int index,
ByteBuffer dst) |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
getBytes(int index,
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf dst,
int dstIndex,
int length) |
int |
getBytes(int index,
FileChannel out,
long position,
int length) |
int |
getBytes(int index,
GatheringByteChannel out,
int length) |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
getBytes(int index,
OutputStream out,
int length) |
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 underlying
MemorySegment . |
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 new
ByteBuffer instance wrapping this buffer's bytes. |
ByteBuffer |
getNioBufferReadable()
Gets a new
ByteBuffer instance wrapping this buffer's readable bytes, i.e. between
Buffer.getReaderIndex() and Buffer.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 |
hasArray() |
boolean |
hasMemoryAddress() |
ByteBuffer |
internalNioBuffer(int index,
int length) |
boolean |
isBuffer()
Returns whether this buffer represents a buffer or an event.
|
boolean |
isCompressed() |
boolean |
isDirect() |
boolean |
isRecycled()
Returns whether this buffer has been recycled or not.
|
long |
memoryAddress() |
ByteBuffer |
nioBuffer(int index,
int length) |
int |
nioBufferCount() |
ByteBuffer[] |
nioBuffers(int index,
int length) |
ByteOrder |
order() |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
readBytes(int length) |
ReadOnlySlicedNetworkBuffer |
readOnlySlice()
Returns a read-only slice of this buffer's readable bytes, i.e. between
Buffer.getReaderIndex() and Buffer.getSize() . |
ReadOnlySlicedNetworkBuffer |
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.
|
NetworkBuffer |
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.
|
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
setBytes(int index,
byte[] src,
int srcIndex,
int length) |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
setBytes(int index,
ByteBuffer src) |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
setBytes(int index,
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf src,
int srcIndex,
int length) |
int |
setBytes(int index,
FileChannel in,
long position,
int length) |
int |
setBytes(int index,
InputStream in,
int length) |
int |
setBytes(int index,
ScatteringByteChannel in,
int length) |
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.
|
String |
toString() |
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
unwrap() |
refCnt, release, release, resetRefCnt, retain, retain, setRefCnt, touch, touch
adjustMarkers, asReadOnly, bytesBefore, bytesBefore, bytesBefore, checkDstIndex, checkDstIndex, checkIndex, checkIndex, checkNewCapacity, checkReadableBytes, checkSrcIndex, clear, compareTo, copy, discardReadBytes, discardSomeReadBytes, duplicate, ensureAccessible, ensureWritable, ensureWritable, equals, forEachByte, forEachByte, forEachByteDesc, forEachByteDesc, getBoolean, getByte, getBytes, getBytes, getBytes, getChar, getCharSequence, getDouble, getFloat, getInt, getIntLE, getLong, getLongLE, getMedium, getMediumLE, getShort, getShortLE, getUnsignedByte, getUnsignedInt, getUnsignedIntLE, getUnsignedMedium, getUnsignedMediumLE, getUnsignedShort, getUnsignedShortLE, hashCode, indexOf, isReadable, isReadable, isReadOnly, isWritable, isWritable, markReaderIndex, markWriterIndex, maxCapacity, maxCapacity, maxWritableBytes, newSwappedByteBuf, nioBuffer, nioBuffers, order, readableBytes, readBoolean, readByte, readBytes, readBytes, readBytes, readBytes, readBytes, readBytes, readBytes, readBytes, readBytes, readChar, readCharSequence, readDouble, readerIndex, readerIndex, readFloat, readInt, readIntLE, readLong, readLongLE, readMedium, readMediumLE, readRetainedSlice, readShort, readShortLE, readSlice, readUnsignedByte, readUnsignedInt, readUnsignedIntLE, readUnsignedMedium, readUnsignedMediumLE, readUnsignedShort, readUnsignedShortLE, resetReaderIndex, resetWriterIndex, retainedDuplicate, retainedSlice, retainedSlice, setBoolean, setByte, setBytes, setBytes, setBytes, setChar, setCharSequence, setDouble, setFloat, setIndex, setInt, setIntLE, setLong, setLongLE, setMedium, setMediumLE, setShort, setShortLE, setZero, skipBytes, slice, slice, toString, toString, trimIndicesToCapacity, writableBytes, writeBoolean, writeByte, writeBytes, writeBytes, writeBytes, writeBytes, writeBytes, writeBytes, writeBytes, writeBytes, writeBytes, writeChar, writeCharSequence, writeDouble, writeFloat, writeInt, writeIntLE, writeLong, writeLongLE, writeMedium, writeMediumLE, writerIndex, writerIndex, writeShort, writeShortLE, writeZero
getDoubleLE, getFloatLE, isContiguous, maxFastWritableBytes, readDoubleLE, readFloatLE, setDoubleLE, setFloatLE, writeDoubleLE, writeFloatLE
clone, finalize, getClass, notify, notifyAll, wait, wait, wait
readableBytes, refCnt, toDebugString
public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler)
memorySegment
- backing memory segment (defines AbstractByteBuf.maxCapacity
)recycler
- will be called to recycle this buffer once the reference count is 0public NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, Buffer.DataType dataType)
memorySegment
- backing memory segment (defines AbstractByteBuf.maxCapacity
)recycler
- will be called to recycle this buffer once the reference count is 0dataType
- the DataType
this buffer representspublic NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, Buffer.DataType dataType, int size)
memorySegment
- backing memory segment (defines AbstractByteBuf.maxCapacity
)recycler
- will be called to recycle this buffer once the reference count is 0dataType
- the DataType
this buffer representssize
- current size of data in the buffer, i.e. the writer index to setpublic NetworkBuffer(MemorySegment memorySegment, BufferRecycler recycler, Buffer.DataType dataType, boolean isCompressed, int size)
memorySegment
- backing memory segment (defines AbstractByteBuf.maxCapacity
)recycler
- will be called to recycle this buffer once the reference count is 0dataType
- the DataType
this buffer representssize
- current size of data in the buffer, i.e. the writer index to setisCompressed
- whether the buffer is compressed or notpublic boolean isBuffer()
Buffer
public MemorySegment getMemorySegment()
Buffer
Buffer.getMemorySegmentOffset()
.
This method will be removed in the future. For writing use BufferBuilder
.
getMemorySegment
in interface Buffer
public int getMemorySegmentOffset()
Buffer
BufferBuilder
.getMemorySegmentOffset
in interface Buffer
Buffer
's data start in the underlying
memory segment.public BufferRecycler getRecycler()
Buffer
getRecycler
in interface Buffer
public void setRecycler(BufferRecycler bufferRecycler)
Buffer
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.
setRecycler
in interface Buffer
bufferRecycler
- the new buffer recyclerpublic void recycleBuffer()
Buffer
recycleBuffer
in interface Buffer
Buffer.retainBuffer()
public boolean isRecycled()
Buffer
isRecycled
in interface Buffer
public NetworkBuffer retainBuffer()
Buffer
retainBuffer
in interface Buffer
Buffer.recycleBuffer()
public ReadOnlySlicedNetworkBuffer readOnlySlice()
Buffer
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.
readOnlySlice
in interface Buffer
public ReadOnlySlicedNetworkBuffer readOnlySlice(int index, int length)
Buffer
Reader and writer indices as well as markers are not shared. Reference counters are shared
but the slice is not retained
automatically.
readOnlySlice
in interface Buffer
index
- the index to start fromlength
- the length of the sliceprotected void deallocate()
deallocate
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCountedByteBuf
protected byte _getByte(int index)
_getByte
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected short _getShort(int index)
_getShort
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected short _getShortLE(int index)
_getShortLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected int _getUnsignedMedium(int index)
_getUnsignedMedium
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected int _getUnsignedMediumLE(int index)
_getUnsignedMediumLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected int _getInt(int index)
_getInt
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected int _getIntLE(int index)
_getIntLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected long _getLong(int index)
_getLong
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected long _getLongLE(int index)
_getLongLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setByte(int index, int value)
_setByte
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setShort(int index, int value)
_setShort
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setShortLE(int index, int value)
_setShortLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setMedium(int index, int value)
_setMedium
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setMediumLE(int index, int value)
_setMediumLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setInt(int index, int value)
_setInt
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setIntLE(int index, int value)
_setIntLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setLong(int index, long value)
_setLong
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
protected void _setLongLE(int index, long value)
_setLongLE
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
public int capacity()
capacity
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public int getMaxCapacity()
Buffer
MemorySegment
.getMaxCapacity
in interface Buffer
public int getReaderIndex()
Buffer
This is where readable (unconsumed) bytes start in the backing memory segment.
getReaderIndex
in interface Buffer
MemorySegment
(inclusive))public void setReaderIndex(int readerIndex) throws IndexOutOfBoundsException
Buffer
setReaderIndex
in interface Buffer
IndexOutOfBoundsException
- if the index is less than 0 or greater than Buffer.getSize()
public int getSize()
Buffer
This is where writable bytes start in the backing memory segment.
getSize
in interface Buffer
MemorySegment
(inclusive))public void setSize(int writerIndex)
Buffer
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf capacity(int newCapacity)
capacity
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public ByteOrder order()
order
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf unwrap()
unwrap
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public boolean isDirect()
isDirect
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf getBytes(int index, org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf dst, int dstIndex, int length)
getBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length)
getBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf getBytes(int index, ByteBuffer dst)
getBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf getBytes(int index, OutputStream out, int length) throws IOException
getBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
IOException
public int getBytes(int index, GatheringByteChannel out, int length) throws IOException
getBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
IOException
public int getBytes(int index, FileChannel out, long position, int length) throws IOException
getBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
IOException
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf setBytes(int index, org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf src, int srcIndex, int length)
setBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf setBytes(int index, byte[] src, int srcIndex, int length)
setBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf setBytes(int index, ByteBuffer src)
setBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public int setBytes(int index, InputStream in, int length) throws IOException
setBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
IOException
public int setBytes(int index, ScatteringByteChannel in, int length) throws IOException
setBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
IOException
public int setBytes(int index, FileChannel in, long position, int length) throws IOException
setBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
IOException
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator alloc()
alloc
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public void setAllocator(org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator allocator)
Buffer
setAllocator
in interface Buffer
allocator
- netty buffer allocatorpublic org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf copy(int index, int length)
copy
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf readBytes(int length)
readBytes
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
public int nioBufferCount()
nioBufferCount
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public ByteBuffer getNioBufferReadable()
Buffer
ByteBuffer
instance wrapping this buffer's readable bytes, i.e. between
Buffer.getReaderIndex()
and Buffer.getSize()
.
Please note that neither index is updated by the returned buffer.
getNioBufferReadable
in interface Buffer
public ByteBuffer getNioBuffer(int index, int length)
Buffer
ByteBuffer
instance wrapping this buffer's bytes.
Please note that neither read nor write index are updated by the returned buffer.
getNioBuffer
in interface Buffer
Buffer.getNioBufferReadable()
public ByteBuffer nioBuffer(int index, int length)
nioBuffer
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public ByteBuffer internalNioBuffer(int index, int length)
internalNioBuffer
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public ByteBuffer[] nioBuffers(int index, int length)
nioBuffers
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public boolean hasArray()
hasArray
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public byte[] array()
array
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public int arrayOffset()
arrayOffset
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public boolean hasMemoryAddress()
hasMemoryAddress
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public long memoryAddress()
memoryAddress
in class org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf
public String toString()
toString
in class org.apache.flink.shaded.netty4.io.netty.buffer.AbstractByteBuf
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf asByteBuf()
public boolean isCompressed()
isCompressed
in interface Buffer
public void setCompressed(boolean isCompressed)
Buffer
setCompressed
in interface Buffer
public Buffer.DataType getDataType()
Buffer
getDataType
in interface Buffer
public void setDataType(Buffer.DataType dataType)
Buffer
setDataType
in interface Buffer
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.