public class NetworkBufferPool extends Object implements BufferPoolFactory, MemorySegmentProvider, AvailabilityProvider
MemorySegment
instances for the network
stack.
The NetworkBufferPool creates LocalBufferPool
s from which the individual tasks draw
the buffers for the network data transfer. When new local buffer pools are created, the
NetworkBufferPool dynamically redistributes the buffers between the pools.
AvailabilityProvider.AvailabilityHelper
Modifier and Type | Field and Description |
---|---|
static int |
UNBOUNDED_POOL_SIZE |
AVAILABLE
Constructor and Description |
---|
NetworkBufferPool(int numberOfSegmentsToAllocate,
int segmentSize) |
NetworkBufferPool(int numberOfSegmentsToAllocate,
int segmentSize,
Duration requestSegmentsTimeout)
Allocates all
MemorySegment instances managed by this pool. |
Modifier and Type | Method and Description |
---|---|
int |
countBuffers() |
BufferPool |
createBufferPool(int numRequiredBuffers,
int maxUsedBuffers)
Tries to create a buffer pool, which is guaranteed to provide at least the number of required
buffers.
|
BufferPool |
createBufferPool(int numRequiredBuffers,
int maxUsedBuffers,
int numSubpartitions,
int maxBuffersPerChannel,
int maxOverdraftBuffersPerGate)
Tries to create a buffer pool with an owner, which is guaranteed to provide at least the
number of required buffers.
|
void |
destroy() |
void |
destroyAllBufferPools()
Destroys all buffer pools that allocate their buffers from this buffer pool (created via
createBufferPool(int, int) ). |
void |
destroyBufferPool(BufferPool bufferPool)
Destroy callback for updating factory book keeping.
|
CompletableFuture<?> |
getAvailableFuture()
Returns a future that is completed when there are free segments in this pool.
|
long |
getAvailableMemory() |
long |
getEstimatedNumberOfRequestedMemorySegments() |
long |
getEstimatedRequestedMemory() |
int |
getEstimatedRequestedSegmentsUsage() |
int |
getNumberOfAvailableMemorySegments() |
int |
getNumberOfRegisteredBufferPools() |
int |
getNumberOfUsedMemorySegments() |
long |
getTotalMemory() |
int |
getTotalNumberOfMemorySegments() |
long |
getUsedMemory() |
boolean |
isDestroyed() |
void |
maybeLogUsageWarning() |
void |
recyclePooledMemorySegment(MemorySegment segment)
Corresponding to
requestPooledMemorySegmentsBlocking(int) and requestPooledMemorySegment() , this method is for pooled memory segments recycling. |
void |
recycleUnpooledMemorySegments(Collection<MemorySegment> segments)
Corresponding to
requestUnpooledMemorySegments(int) , this method is for unpooled memory
segments recycling. |
MemorySegment |
requestPooledMemorySegment()
Different from
requestUnpooledMemorySegments(int) for unpooled segments allocation. |
List<MemorySegment> |
requestPooledMemorySegmentsBlocking(int numberOfSegmentsToRequest) |
List<MemorySegment> |
requestUnpooledMemorySegments(int numberOfSegmentsToRequest)
Unpooled memory segments are requested directly from
NetworkBufferPool , as opposed to
pooled segments, that are requested through BufferPool that was created from this
NetworkBufferPool (see createBufferPool(int, int) ). |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
and, isApproximatelyAvailable, isAvailable, or
public static final int UNBOUNDED_POOL_SIZE
@VisibleForTesting public NetworkBufferPool(int numberOfSegmentsToAllocate, int segmentSize)
public NetworkBufferPool(int numberOfSegmentsToAllocate, int segmentSize, Duration requestSegmentsTimeout)
MemorySegment
instances managed by this pool.@Nullable public MemorySegment requestPooledMemorySegment()
requestUnpooledMemorySegments(int)
for unpooled segments allocation. This
method and the below requestPooledMemorySegmentsBlocking(int)
method are designed to be
used from LocalBufferPool
for pooled memory segments allocation. Note that these
methods for pooled memory segments requesting and recycling are prohibited from acquiring the
factoryLock to avoid deadlock.public List<MemorySegment> requestPooledMemorySegmentsBlocking(int numberOfSegmentsToRequest) throws IOException
IOException
public void recyclePooledMemorySegment(MemorySegment segment)
requestPooledMemorySegmentsBlocking(int)
and requestPooledMemorySegment()
, this method is for pooled memory segments recycling.public List<MemorySegment> requestUnpooledMemorySegments(int numberOfSegmentsToRequest) throws IOException
NetworkBufferPool
, as opposed to
pooled segments, that are requested through BufferPool
that was created from this
NetworkBufferPool
(see createBufferPool(int, int)
). They are used for example for
exclusive RemoteInputChannel
credits, that are permanently assigned to that channel,
and never returned to any BufferPool
. As opposed to pooled segments, when requested,
unpooled segments needs to be accounted against numTotalRequiredBuffers
, which might
require redistribution of the segments.requestUnpooledMemorySegments
in interface MemorySegmentProvider
IOException
public void recycleUnpooledMemorySegments(Collection<MemorySegment> segments)
requestUnpooledMemorySegments(int)
, this method is for unpooled memory
segments recycling.recycleUnpooledMemorySegments
in interface MemorySegmentProvider
public void destroy()
public boolean isDestroyed()
public int getTotalNumberOfMemorySegments()
public long getTotalMemory()
public int getNumberOfAvailableMemorySegments()
public long getAvailableMemory()
public int getNumberOfUsedMemorySegments()
public long getUsedMemory()
public int getNumberOfRegisteredBufferPools()
public long getEstimatedNumberOfRequestedMemorySegments()
public long getEstimatedRequestedMemory()
public int getEstimatedRequestedSegmentsUsage()
public void maybeLogUsageWarning()
public int countBuffers()
public CompletableFuture<?> getAvailableFuture()
getAvailableFuture
in interface AvailabilityProvider
public BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers) throws IOException
BufferPoolFactory
The buffer pool is of dynamic size with at least numRequiredBuffers buffers.
createBufferPool
in interface BufferPoolFactory
numRequiredBuffers
- minimum number of network buffers in this poolmaxUsedBuffers
- maximum number of network buffers this pool offersIOException
public BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers, int numSubpartitions, int maxBuffersPerChannel, int maxOverdraftBuffersPerGate) throws IOException
BufferPoolFactory
The buffer pool is of dynamic size with at least numRequiredBuffers buffers.
createBufferPool
in interface BufferPoolFactory
numRequiredBuffers
- minimum number of network buffers in this poolmaxUsedBuffers
- maximum number of network buffers this pool offersnumSubpartitions
- number of subpartitions in this poolmaxBuffersPerChannel
- maximum number of buffers to use for each channelmaxOverdraftBuffersPerGate
- maximum number of overdraft buffers to use for each gateIOException
public void destroyBufferPool(BufferPool bufferPool)
BufferPoolFactory
destroyBufferPool
in interface BufferPoolFactory
public void destroyAllBufferPools()
createBufferPool(int, int)
).Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.