public class FileRegionBuffer extends org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion implements Buffer
FileChannel.transferTo(long, long, WritableByteChannel)
.
This class implements Buffer
mainly for compatible with existing usages. It can be
thought of as a "lazy buffer" that does not hold the data directly, although the data can be
fetches as a read-only ByteBuffer
when needed, for example in local input channels. See
readInto(MemorySegment)
and getNioBufferReadable()
. Because this buffer is
read-only, the modification methods (and methods that give a writable buffer) throw UnsupportedOperationException
.
This class extends from Netty's DefaultFileRegion
, similar as the NetworkBuffer
extends from Netty's ByteBuf
. That way we can pass both of them to Netty
in the same way, and Netty will internally treat them appropriately.
Buffer.DataType
Constructor and Description |
---|
FileRegionBuffer(FileChannel fileChannel,
long fileChannelPosition,
int bufferSize,
Buffer.DataType dataType,
boolean isCompressed) |
Modifier and Type | Method and Description |
---|---|
org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf |
asByteBuf() |
void |
deallocate() |
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()
This method is only called by tests and by event-deserialization, like checkpoint barriers.
|
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 as
Buffer.getSize() - Buffer.getReaderIndex() ). |
Buffer |
readInto(MemorySegment segment) |
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.
|
FileRegionBuffer |
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.
|
count, isOpen, open, position, retain, retain, touch, touch, transfered, transferred, transferTo
refCnt, release, release, setRefCnt
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
refCnt, toDebugString
public FileRegionBuffer(FileChannel fileChannel, long fileChannelPosition, int bufferSize, Buffer.DataType dataType, boolean isCompressed)
public 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 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 slicepublic 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 ByteBuffer getNioBufferReadable()
getNioBufferReadable
in interface Buffer
public ByteBuffer getNioBuffer(int index, int length) throws IndexOutOfBoundsException
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
IndexOutOfBoundsException
- if the indexes are not without the buffer's boundsBuffer.getNioBufferReadable()
public org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf asByteBuf()
public void setSize(int writerIndex)
Buffer
public int getSize()
Buffer
This is where writable bytes start in the backing memory segment.
getSize
in interface Buffer
MemorySegment
(inclusive))public int readableBytes()
Buffer
Buffer.getSize()
- Buffer.getReaderIndex()
).readableBytes
in interface Buffer
public void setAllocator(org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator allocator)
Buffer
setAllocator
in interface Buffer
allocator
- netty buffer allocatorpublic 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 FileRegionBuffer retainBuffer()
Buffer
retainBuffer
in interface Buffer
Buffer.recycleBuffer()
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
public void deallocate()
deallocate
in class org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion
public Buffer readInto(MemorySegment segment) throws IOException
IOException
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.