@InterfaceAudience.Private public class ByteBuffAllocator extends Object
ByteBuff
, if we are sure that those ByteBuffers have reached the end of life
cycle, we must do the ByteBuff.release()
to return back the buffers to the pool,
otherwise ByteBuffers leak will happen, and the NIO ByteBuffer pool may be exhausted. there's
possible that the desired memory size is large than ByteBufferPool has, we'll downgrade to
allocate ByteBuffers from heap which meaning the GC pressure may increase again. Of course, an
better way is increasing the ByteBufferPool size if we detected this case. ByteBuff
in the RPC layer or the entire
read/write path, because it hide the details of memory management and its APIs are more friendly
to the upper layer.Modifier and Type | Class and Description |
---|---|
static interface |
ByteBuffAllocator.Recycler |
Modifier and Type | Field and Description |
---|---|
static String |
ALLOCATOR_POOL_ENABLED_KEY |
static String |
BUFFER_SIZE_KEY |
private Queue<ByteBuffer> |
buffers |
protected int |
bufSize |
static String |
BYTEBUFF_ALLOCATOR_CLASS
Set an alternate bytebuffallocator by setting this config, e.g.
|
static int |
DEFAULT_BUFFER_SIZE
There're some reasons why better to choose 65KB(rather than 64KB) as the default buffer size:
|
static String |
DEPRECATED_ALLOCATOR_POOL_ENABLED_KEY
Deprecated.
since 2.3.0 and will be removed in 4.0.0. Use
ALLOCATOR_POOL_ENABLED_KEY instead. |
(package private) static String |
DEPRECATED_BUFFER_SIZE_KEY
Deprecated.
since 2.3.0 and will be removed in 4.0.0. Use
BUFFER_SIZE_KEY instead. |
(package private) static String |
DEPRECATED_MAX_BUFFER_COUNT_KEY
Deprecated.
since 2.3.0 and will be removed in 4.0.0. Use
MAX_BUFFER_COUNT_KEY instead. |
static ByteBuffAllocator |
HEAP |
private LongAdder |
heapAllocationBytes |
private long |
lastHeapAllocationBytes |
private long |
lastPoolAllocationBytes |
private static org.slf4j.Logger |
LOG |
static String |
MAX_BUFFER_COUNT_KEY |
private int |
maxBufCount |
private boolean |
maxPoolSizeInfoLevelLogged |
static String |
MIN_ALLOCATE_SIZE_KEY |
private int |
minSizeForReservoirUse |
static ByteBuffAllocator.Recycler |
NONE |
private LongAdder |
poolAllocationBytes |
protected boolean |
reservoirEnabled |
private AtomicInteger |
usedBufCount |
Modifier | Constructor and Description |
---|---|
protected |
ByteBuffAllocator(boolean reservoirEnabled,
int maxBufCount,
int bufSize,
int minSizeForReservoirUse) |
Modifier and Type | Method and Description |
---|---|
ByteBuff |
allocate(int size)
Allocate size bytes from the ByteBufAllocator, Note to call the
ByteBuff.release() if
no need any more, otherwise the memory leak happen in NIO ByteBuffer pool. |
SingleByteBuff |
allocateOneBuffer()
Allocate an buffer with buffer size from ByteBuffAllocator, Note to call the
ByteBuff.release() if no need any more, otherwise the memory leak happen in NIO
ByteBuffer pool. |
private ByteBuffer |
allocateOnHeap(int size) |
void |
clean()
Free all direct buffers if allocated, mainly used for testing.
|
static ByteBuffAllocator |
create(org.apache.hadoop.conf.Configuration conf,
boolean reservoirEnabled)
Initialize an
ByteBuffAllocator which will try to allocate ByteBuffers from off-heap if
reservoir is enabled and the reservoir has enough buffers, otherwise the allocator will just
allocate the insufficient buffers from on-heap to meet the requirement. |
private static ByteBuffAllocator |
createOnHeap()
Initialize an
ByteBuffAllocator which only allocate ByteBuffer from on-heap, it's
designed for testing purpose or disabled reservoir case. |
private ByteBuffer |
getBuffer() |
int |
getBufferSize() |
int |
getFreeBufferCount()
The
ConcurrentLinkedQueue.size() is O(N) complexity and time-consuming, so DO NOT use
the method except in UT. |
long |
getHeapAllocationBytes() |
static long |
getHeapAllocationBytes(ByteBuffAllocator... allocators) |
static double |
getHeapAllocationRatio(ByteBuffAllocator... allocators) |
long |
getPoolAllocationBytes() |
int |
getTotalBufferCount() |
int |
getUsedBufferCount() |
boolean |
isReservoirEnabled() |
protected void |
putbackBuffer(ByteBuffer buf)
Return back a ByteBuffer after its use.
|
private static final org.slf4j.Logger LOG
public static final ByteBuffAllocator HEAP
public static final String ALLOCATOR_POOL_ENABLED_KEY
public static final String MAX_BUFFER_COUNT_KEY
public static final String BUFFER_SIZE_KEY
public static final String MIN_ALLOCATE_SIZE_KEY
public static final String BYTEBUFF_ALLOCATOR_CLASS
DeallocateRewriteByteBuffAllocator
to find out prematurely release issues@Deprecated public static final String DEPRECATED_ALLOCATOR_POOL_ENABLED_KEY
ALLOCATOR_POOL_ENABLED_KEY
instead.@Deprecated static final String DEPRECATED_MAX_BUFFER_COUNT_KEY
MAX_BUFFER_COUNT_KEY
instead.@Deprecated static final String DEPRECATED_BUFFER_SIZE_KEY
BUFFER_SIZE_KEY
instead.public static final int DEFAULT_BUFFER_SIZE
1. Almost all of the data blocks have the block size: 64KB + delta, whose delta is very small, depends on the size of lastKeyValue. If we set buffer.size=64KB, then each block will be allocated as a MultiByteBuff: one 64KB DirectByteBuffer and delta bytes HeapByteBuffer, the HeapByteBuffer will increase the GC pressure. Ideally, we should let the data block to be allocated as a SingleByteBuff, it has simpler data structure, faster access speed, less heap usage.
2. Since the blocks are MultiByteBuff when using buffer.size=64KB, so we have to calculate the checksum by an temp heap copying (see HBASE-21917), while if it's a SingleByteBuff, we can speed the checksum by calling the hadoop' checksum in native lib, which is more faster.
For performance comparison, please see HBASE-22483.
public static final ByteBuffAllocator.Recycler NONE
protected final boolean reservoirEnabled
protected final int bufSize
private final int maxBufCount
private final AtomicInteger usedBufCount
private boolean maxPoolSizeInfoLevelLogged
private final int minSizeForReservoirUse
private final Queue<ByteBuffer> buffers
private final LongAdder poolAllocationBytes
private final LongAdder heapAllocationBytes
private long lastPoolAllocationBytes
private long lastHeapAllocationBytes
protected ByteBuffAllocator(boolean reservoirEnabled, int maxBufCount, int bufSize, int minSizeForReservoirUse)
public static ByteBuffAllocator create(org.apache.hadoop.conf.Configuration conf, boolean reservoirEnabled)
ByteBuffAllocator
which will try to allocate ByteBuffers from off-heap if
reservoir is enabled and the reservoir has enough buffers, otherwise the allocator will just
allocate the insufficient buffers from on-heap to meet the requirement.conf
- which get the arguments to initialize the allocator.reservoirEnabled
- indicate whether the reservoir is enabled or disabled. NOTICE: if
reservoir is enabled, then we will use the pool allocator to allocate
off-heap ByteBuffers and use the HEAP allocator to allocate heap
ByteBuffers. Otherwise if reservoir is disabled then all allocations
will happen in HEAP instance.private static ByteBuffAllocator createOnHeap()
ByteBuffAllocator
which only allocate ByteBuffer from on-heap, it's
designed for testing purpose or disabled reservoir case.public boolean isReservoirEnabled()
public long getHeapAllocationBytes()
public long getPoolAllocationBytes()
public int getBufferSize()
public int getUsedBufferCount()
public int getFreeBufferCount()
ConcurrentLinkedQueue.size()
is O(N) complexity and time-consuming, so DO NOT use
the method except in UT.public int getTotalBufferCount()
public static long getHeapAllocationBytes(ByteBuffAllocator... allocators)
public static double getHeapAllocationRatio(ByteBuffAllocator... allocators)
public SingleByteBuff allocateOneBuffer()
ByteBuff.release()
if no need any more, otherwise the memory leak happen in NIO
ByteBuffer pool.private ByteBuffer allocateOnHeap(int size)
public ByteBuff allocate(int size)
ByteBuff.release()
if
no need any more, otherwise the memory leak happen in NIO ByteBuffer pool.size
- to allocatepublic void clean()
private ByteBuffer getBuffer()
protected void putbackBuffer(ByteBuffer buf)
buf
- ByteBuffer to return.Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.