@InterfaceAudience.Private public abstract class ByteBuff extends Object implements HBaseReferenceCounted
HBaseReferenceCounted
interface which mean need to maintains a
RefCnt
inside, if ensure that the ByteBuff won't be used any more, we must do a
release()
to recycle its NIO ByteBuffers. when considering the
duplicate()
or slice()
, releasing either the duplicated one or
the original one will free its memory, because they share the same NIO ByteBuffers. when you want
to retain the NIO ByteBuffers even if the origin one called release()
, you can
do like this:
ByteBuff original = ...; ByteBuff dup = original.duplicate(); dup.retain(); original.release(); // The NIO buffers can still be accessed unless you release the duplicated one dup.get(...); dup.release(); // Both the original and dup can not access the NIO buffers any more.
Modifier and Type | Class and Description |
---|---|
(package private) static interface |
ByteBuff.ChannelReader
Functional interface for Channel read
|
Modifier and Type | Field and Description |
---|---|
(package private) static ByteBuff.ChannelReader |
CHANNEL_READER |
(package private) static ByteBuff.ChannelReader |
FILE_READER |
private static int |
NIO_BUFFER_LIMIT |
protected RefCnt |
refCnt |
private static String |
REFERENCE_COUNT_NAME |
Constructor and Description |
---|
ByteBuff() |
Modifier and Type | Method and Description |
---|---|
abstract byte[] |
array()
Returns the byte[] if the underlying BB has single BB and hasArray true
|
abstract int |
arrayOffset()
Returns the arrayOffset of the byte[] incase of a single BB backed ByteBuff
|
abstract ByteBuffer |
asSubByteBuffer(int length)
Returns bytes from current position till length specified, as a single ByteBuffer.
|
abstract void |
asSubByteBuffer(int offset,
int length,
ObjectIntPair<ByteBuffer> pair)
Returns bytes from given offset till length specified, as a single ByteBuffer.
|
abstract int |
capacity()
Returns the total capacity of this ByteBuff.
|
protected void |
checkRefCount()
Methods for reference count
|
static int |
compareTo(ByteBuff buf1,
int o1,
int len1,
ByteBuff buf2,
int o2,
int len2)
Compares two ByteBuffs
|
abstract ByteBuff |
duplicate()
Returns an ByteBuff which is a duplicate version of this ByteBuff.
|
abstract byte |
get()
A relative method that returns byte at the current position.
|
abstract void |
get(byte[] dst)
Copies the content from this ByteBuff's current position to the byte array and fills it.
|
abstract void |
get(byte[] dst,
int offset,
int length)
Copies the specified number of bytes from this ByteBuff's current position to the byte[]'s
offset.
|
abstract void |
get(ByteBuffer out,
int sourceOffset,
int length)
Copies the content from this ByteBuff to a ByteBuffer Note : This will advance the position
marker of
out but not change the position maker for this ByteBuff |
abstract byte |
get(int index)
Fetches the byte at the given index.
|
abstract void |
get(int sourceOffset,
byte[] dst,
int offset,
int length)
Copies the specified number of bytes from this ByteBuff's given position to the byte[]'s
offset.
|
abstract byte |
getByteAfterPosition(int offset)
Fetches the byte at the given offset from current position.
|
abstract int |
getInt()
Returns the int value at the current position.
|
abstract int |
getInt(int index)
Fetches the int at the given index.
|
abstract int |
getIntAfterPosition(int offset)
Fetches the int value at the given offset from current position.
|
abstract long |
getLong()
Returns the long value at the current position.
|
abstract long |
getLong(int index)
Fetches the long at the given index.
|
abstract long |
getLongAfterPosition(int offset)
Fetches the long value at the given offset from current position.
|
RefCnt |
getRefCnt() |
abstract short |
getShort()
Returns the short value at the current position.
|
abstract short |
getShort(int index)
Fetches the short value at the given index.
|
abstract short |
getShortAfterPosition(int offset)
Fetches the short value at the given offset from current position.
|
abstract boolean |
hasArray()
Returns true or false if the underlying BB support hasArray
|
abstract boolean |
hasRemaining()
Returns true if there are elements between the current position and the limit.
|
abstract int |
limit()
Returns the limit of this ByteBuff
|
abstract ByteBuff |
limit(int limit)
Marks the limit of this ByteBuff
|
abstract ByteBuff |
mark()
Marks the current position of the ByteBuff
|
abstract ByteBuff |
moveBack(int len)
Jumps back the current position of this ByteBuff by specified length.
|
abstract ByteBuffer[] |
nioByteBuffers() |
abstract int |
position()
Returns this ByteBuff's current position
|
abstract ByteBuff |
position(int position)
Sets this ByteBuff's position to the given value.
|
abstract ByteBuff |
put(byte b)
Writes a byte to this ByteBuff at the current position and increments the position
|
abstract ByteBuff |
put(byte[] src)
Copies from the given byte[] to this ByteBuff n * @return this ByteBuff
|
abstract ByteBuff |
put(byte[] src,
int offset,
int length)
Copies from the given byte[] to this ByteBuff
|
abstract ByteBuff |
put(int index,
byte b)
Writes a byte to this ByteBuff at the given index
|
abstract ByteBuff |
put(int offset,
ByteBuff src,
int srcOffset,
int length)
Copies the contents from the src ByteBuff to this ByteBuff.
|
abstract ByteBuff |
putInt(int value)
Writes an int to this ByteBuff at its current position.
|
abstract ByteBuff |
putLong(long value)
Writes a long to this ByteBuff at its current position.
|
abstract int |
read(FileChannel channel,
long offset)
Reads bytes from FileChannel into this ByteBuff
|
abstract int |
read(ReadableByteChannel channel)
Reads bytes from the given channel into this ByteBuf.
|
static int |
read(ReadableByteChannel channel,
ByteBuffer buf,
long offset,
ByteBuff.ChannelReader reader) |
static int |
readCompressedInt(ByteBuff buf)
Read integer from ByteBuff coded in 7 bits and increment position.
|
static long |
readLong(ByteBuff in,
int fitInBytes)
Read long which was written to fitInBytes bytes and increment position.
|
int |
refCnt() |
boolean |
release() |
abstract int |
remaining()
Returns the number of elements between the current position and the limit.
|
abstract ByteBuff |
reset()
Similar to
ByteBuffer .reset(), ensures that this ByteBuff is reset back to last marked
position. |
abstract ByteBuff |
rewind()
Rewinds this ByteBuff and the position is set to 0
|
abstract ByteBuff |
skip(int len)
Jumps the current position of this ByteBuff by specified length.
|
abstract ByteBuff |
slice()
Returns an ByteBuff which is a sliced version of this ByteBuff.
|
byte[] |
toBytes()
Copy the content from this ByteBuff to a byte[].
|
abstract byte[] |
toBytes(int offset,
int length)
Copy the content from this ByteBuff to a byte[] based on the given offset and length.
|
String |
toString() |
ByteBuff |
touch()
Calling this method in strategic locations where ByteBuffs are referenced may help diagnose
potential buffer leaks.
|
ByteBuff |
touch(Object hint) |
static ByteBuff |
wrap(ByteBuffer buffer) |
static ByteBuff |
wrap(ByteBuffer[] buffers) |
static ByteBuff |
wrap(ByteBuffer[] buffers,
ByteBuffAllocator.Recycler recycler) |
static ByteBuff |
wrap(ByteBuffer[] buffers,
RefCnt refCnt)
In theory, the upstream should never construct an ByteBuff by passing an given refCnt, so
please don't use this public method in other place.
|
private static ByteBuff |
wrap(ByteBuffer buffer,
RefCnt refCnt) |
static ByteBuff |
wrap(List<ByteBuffer> buffers) |
static ByteBuff |
wrap(List<ByteBuffer> buffers,
ByteBuffAllocator.Recycler recycler) |
private static ByteBuff |
wrap(List<ByteBuffer> buffers,
RefCnt refCnt) |
abstract int |
write(FileChannel channel,
long offset)
Write this ByteBuff's data into target file
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
release, retain
private static final String REFERENCE_COUNT_NAME
private static final int NIO_BUFFER_LIMIT
static final ByteBuff.ChannelReader CHANNEL_READER
static final ByteBuff.ChannelReader FILE_READER
public ByteBuff()
protected void checkRefCount()
public int refCnt()
refCnt
in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
public boolean release()
release
in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
public abstract int position()
public abstract ByteBuff position(int position)
public abstract ByteBuff skip(int len)
len
- the length to be skippedpublic abstract ByteBuff moveBack(int len)
len
- the length to move backpublic abstract int capacity()
public abstract int limit()
public abstract ByteBuffer asSubByteBuffer(int length)
length
- number of bytes required.public abstract void asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair)
offset
- the offset in this ByteBuff from where the subBuffer should be createdlength
- the length of the subBufferpair
- a pair that will have the bytes from the current position till length specified,
as a single ByteBuffer and offset in that Buffer where the bytes starts. Since
this API gets called in a loop we are passing a pair to it which could be created
outside the loop and the method would set the values on the pair that is passed
in by the caller. Thus it avoids more object creations that would happen if the
pair that is returned is created by this method every time.public abstract int remaining()
public abstract boolean hasRemaining()
public abstract ByteBuff reset()
ByteBuffer
.reset(), ensures that this ByteBuff is reset back to last marked
position.public abstract ByteBuff slice()
public abstract ByteBuff duplicate()
public abstract byte get()
public abstract byte get(int index)
public abstract byte getByteAfterPosition(int offset)
public abstract ByteBuff put(byte b)
public abstract ByteBuff put(int index, byte b)
public abstract void get(byte[] dst, int offset, int length)
dst
- the byte[] to which the ByteBuff's content is to be copiedoffset
- within the current arraylength
- upto which the bytes to be copiedpublic abstract void get(int sourceOffset, byte[] dst, int offset, int length)
sourceOffset
- the offset in this ByteBuff from where the copy should happendst
- the byte[] to which the ByteBuff's content is to be copiedoffset
- within the current arraylength
- upto which the bytes to be copiedpublic abstract void get(byte[] dst)
dst
- the byte[] to which the ByteBuff's content is to be copiedpublic abstract ByteBuff put(byte[] src, int offset, int length)
src
- source byte arrayoffset
- the position in the byte array from which the copy should be donelength
- the length upto which the copy should happenpublic abstract ByteBuff put(byte[] src)
src
- source byte arraypublic abstract boolean hasArray()
public abstract byte[] array()
public abstract int arrayOffset()
public abstract short getShort()
public abstract short getShort(int index)
public abstract short getShortAfterPosition(int offset)
public abstract int getInt()
public abstract ByteBuff putInt(int value)
public abstract int getInt(int index)
public abstract int getIntAfterPosition(int offset)
public abstract long getLong()
public abstract ByteBuff putLong(long value)
public abstract long getLong(int index)
public abstract long getLongAfterPosition(int offset)
public byte[] toBytes()
public abstract byte[] toBytes(int offset, int length)
public abstract void get(ByteBuffer out, int sourceOffset, int length)
out
but not change the position maker for this ByteBuffout
- the ByteBuffer to which the copy has to happensourceOffset
- the offset in the ByteBuff from which the elements has to be copiedlength
- the length in this ByteBuff upto which the elements has to be copiedpublic abstract ByteBuff put(int offset, ByteBuff src, int srcOffset, int length)
offset
- the position in this ByteBuff to which the copy should happensrc
- the src ByteBuffsrcOffset
- the offset in the src ByteBuff from where the elements should be readlength
- the length up to which the copy should happenpublic abstract int read(ReadableByteChannel channel) throws IOException
IOException
public abstract int read(FileChannel channel, long offset) throws IOException
IOException
public abstract int write(FileChannel channel, long offset) throws IOException
IOException
public static int read(ReadableByteChannel channel, ByteBuffer buf, long offset, ByteBuff.ChannelReader reader) throws IOException
IOException
public static int readCompressedInt(ByteBuff buf)
public static int compareTo(ByteBuff buf1, int o1, int len1, ByteBuff buf2, int o2, int len2)
buf1
- the first ByteBuffo1
- the offset in the first ByteBuff from where the compare has to happenlen1
- the length in the first ByteBuff upto which the compare has to happenbuf2
- the second ByteBuffo2
- the offset in the second ByteBuff from where the compare has to happenlen2
- the length in the second ByteBuff upto which the compare has to happenpublic static long readLong(ByteBuff in, int fitInBytes)
fitInBytes
- In how many bytes given long is stored.public abstract ByteBuffer[] nioByteBuffers()
public static ByteBuff wrap(ByteBuffer[] buffers, RefCnt refCnt)
public static ByteBuff wrap(ByteBuffer[] buffers, ByteBuffAllocator.Recycler recycler)
public static ByteBuff wrap(ByteBuffer[] buffers)
public static ByteBuff wrap(List<ByteBuffer> buffers, ByteBuffAllocator.Recycler recycler)
public static ByteBuff wrap(List<ByteBuffer> buffers)
public static ByteBuff wrap(ByteBuffer buffer)
public ByteBuff touch()
touch(Object)
to pass their own hint as well.touch
in interface HBaseReferenceCounted
touch
in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
public ByteBuff touch(Object hint)
touch
in interface HBaseReferenceCounted
touch
in interface org.apache.hbase.thirdparty.io.netty.util.ReferenceCounted
private static ByteBuff wrap(List<ByteBuffer> buffers, RefCnt refCnt)
private static ByteBuff wrap(ByteBuffer buffer, RefCnt refCnt)
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.