zhuzhurk commented on a change in pull request #15199:
URL: https://github.com/apache/flink/pull/15199#discussion_r599248454
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java
##########
@@ -50,6 +52,12 @@
private NetworkBufferPool networkBufferPool = new NetworkBufferPool(2, 1);
+ private BatchShuffleReadBufferPool batchShuffleReadBufferPool =
+ new BatchShuffleReadBufferPool(1024, 32 * 1024);
Review comment:
why set a `totalBytes` to be smaller than `bufferSize` in this default
batchShuffleReadBufferPool? Seems it will lead to exceptions if used.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPool.java
##########
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A fixed-size {@link MemorySegment} pool used by batch shuffle for shuffle
data read (currently
+ * only used by sort-merge blocking shuffle).
+ */
+public class BatchShuffleReadBufferPool {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(BatchShuffleReadBufferPool.class);
+
+ /** Minimum total memory size in bytes of this buffer pool. */
+ public static final int MIN_TOTAL_BYTES = 32 * 1024 * 1024;
+
+ /**
+ * Memory size in bytes can be allocated from this buffer pool for a
single request (8M is for
+ * better sequential read).
+ */
+ public static final int NUM_BYTES_PER_REQUEST = 8 * 1024 * 1024;
+
+ /** Total direct memory size in bytes can can be allocated and used by
this buffer pool. */
+ private final long totalBytes;
+
+ /**
+ * Maximum time to wait in milliseconds when requesting read buffers from
this buffer pool
+ * before throwing an exception.
+ */
+ private final long requestTimeout;
+
+ /** The number of total buffers in this buffer pool. */
+ private final int numTotalBuffers;
+
+ /** Size of each buffer in bytes in this buffer pool. */
+ private final int bufferSize;
+
+ /** The number of buffers to be returned for a single request. */
+ private final int numBuffersPerRequest;
+
+ /** All available buffers in this buffer pool currently. */
+ @GuardedBy("buffers")
+ private final Queue<MemorySegment> buffers = new ArrayDeque<>();
+
+ /** Whether this buffer pool has been destroyed or not. */
+ @GuardedBy("buffers")
+ private boolean destroyed;
+
+ /** Whether this buffer pool has been initialized or not. */
+ @GuardedBy("buffers")
+ private boolean initialized;
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize) {
+ // 5 min default buffer request timeout
+ this(totalBytes, bufferSize, 5 * 60 * 1000);
+ }
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize, long
requestTimeout) {
+ checkArgument(totalBytes > 0, "Total memory size must be positive.");
+ checkArgument(bufferSize > 0, "Size of buffer must be positive.");
+ checkArgument(requestTimeout > 0, "Request timeout must be positive.");
+
+ this.totalBytes = totalBytes;
+ this.bufferSize = bufferSize;
+ this.requestTimeout = requestTimeout;
+
+ this.numTotalBuffers = (int) Math.min(totalBytes / bufferSize,
Integer.MAX_VALUE);
+ this.numBuffersPerRequest =
+ Math.min(numTotalBuffers, Math.max(1, NUM_BYTES_PER_REQUEST /
bufferSize));
+ }
+
+ @VisibleForTesting
+ public long getTotalBytes() {
+ return totalBytes;
+ }
+
+ @VisibleForTesting
+ public int getNumBuffersPerRequest() {
+ return numBuffersPerRequest;
+ }
+
+ @VisibleForTesting
+ public int getNumTotalBuffers() {
+ return numTotalBuffers;
+ }
+
+ @VisibleForTesting
+ public int getAvailableBuffers() {
+ synchronized (buffers) {
+ return buffers.size();
+ }
+ }
+
+ public int getMaxConcurrentRequests() {
+ return numBuffersPerRequest > 0 ? numTotalBuffers /
numBuffersPerRequest : 0;
+ }
+
+ /** Initializes this buffer pool which allocates all the buffers. */
+ public void initialize() {
+ LOG.info(
+ "Initializing batch shuffle IO buffer pool: numBuffers={},
bufferSize={}.",
+ numTotalBuffers,
+ bufferSize);
+
+ checkState(
+ totalBytes >= bufferSize,
+ String.format(
+ "Illegal configuration, config value for %s must be no
smaller than %s,"
+ + " please increase %s to at least %d bytes.",
+
TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key(),
+ TaskManagerOptions.MEMORY_SEGMENT_SIZE.key(),
+
TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key(),
+ bufferSize));
+
+ synchronized (buffers) {
+ checkState(!destroyed, "Buffer pool is already destroyed.");
+
+ if (initialized) {
+ return;
+ }
+ initialized = true;
+
+ try {
+ for (int i = 0; i < numTotalBuffers; ++i) {
+
buffers.add(MemorySegmentFactory.allocateUnpooledOffHeapMemory(bufferSize));
+ }
+ } catch (OutOfMemoryError outOfMemoryError) {
+ int allocated = buffers.size();
+ buffers.clear();
+ throw new OutOfMemoryError(
+ String.format(
+ "Can't allocate enough direct buffer for batch
shuffle read buffer "
+ + "pool (bytes allocated: %d, bytes
still needed: %d). To "
+ + "avoid the exception, you need to do
one of the following"
+ + " adjustments: 1) If you have ever
decreased %s, you need"
+ + " to undo the decrement; 2) If you
ever increased %s, you"
+ + " should also increase %s; 3) If
neither the above cases,"
+ + " it usually means some other parts
of your application "
+ + "have consumed too many direct
memory and the value of %s"
+ + " should be increased.",
+ allocated * bufferSize,
+ (numTotalBuffers - allocated) * bufferSize,
+
TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY.key(),
+
TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key(),
+
TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY.key(),
+
TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key()));
+ }
+ }
+ }
+
+ /**
+ * Requests a collection of buffers (determined by {@link
#numBuffersPerRequest}) from this
+ * buffer pool. Exception will be thrown if no enough buffers can be
allocated in the given
+ * timeout and the corresponding requester is not holding any allocated
buffers currently.
+ */
+ public List<MemorySegment> requestBuffers(Object owner) throws Exception {
+ checkArgument(owner != null, "Owner must be not null.");
+
+ List<MemorySegment> allocated = new ArrayList<>(numBuffersPerRequest);
+ synchronized (buffers) {
+ checkState(!destroyed, "Buffer pool is already destroyed.");
+
+ if (!initialized) {
+ initialize();
+ }
+
+ long startTime = System.currentTimeMillis();
+ while (buffers.size() < numBuffersPerRequest) {
+ checkState(!destroyed, "Buffer pool is already destroyed.");
+
+ buffers.wait(requestTimeout);
+ if (System.currentTimeMillis() - startTime >= requestTimeout) {
Review comment:
seems the timeout will never trigger if available `buffers` does not
increase to be larger than `numBuffersPerRequest`, because `buffers` is only
notified when it can fulfill a request.
I feel that we should not let the timeout rely on buffer returning events,
because in the worst case that no buffer can be returned, timeout will never be
triggered.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A fixed-size {@link MemorySegment} pool used by batch shuffle for shuffle
data read (currently
+ * only used by sort-merge blocking shuffle).
+ */
+@Internal
+public class BatchShuffleReadBufferPool {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(BatchShuffleReadBufferPool.class);
+
+ /** Minimum total memory size in bytes of this buffer pool. */
+ public static final int MIN_TOTAL_BYTES = 32 * 1024 * 1024;
+
+ /**
+ * Memory size in bytes can be allocated from this buffer pool for a
single request (8M is for
+ * better sequential read).
+ */
+ public static final int NUM_BYTES_PER_REQUEST = 8 * 1024 * 1024;
+
+ /** Total direct memory size in bytes can can be allocated and used by
this buffer pool. */
+ private final long totalBytes;
+
+ /**
+ * Maximum time to wait in milliseconds when requesting read buffers from
this buffer pool
+ * before throwing an exception.
+ */
+ private final long requestTimeout;
+
+ /** The number of total buffers in this buffer pool. */
+ private final int numTotalBuffers;
+
+ /** Size of each buffer in bytes in this buffer pool. */
+ private final int bufferSize;
+
+ /** The number of buffers to be returned for a single request. */
+ private final int numBuffersPerRequest;
+
+ /**
+ * The maximum number of buffers can be allocated from this buffer pool
for a single buffer
+ * requester.
+ */
+ private final int maxBuffersPerRequester;
+
+ /** All available buffers in this buffer pool currently. */
+ @GuardedBy("buffers")
+ private final Queue<MemorySegment> buffers = new ArrayDeque<>();
+
+ /** Account for all the buffers requested per requester. */
+ @GuardedBy("buffers")
+ private final Map<Object, Counter> numBuffersAllocated = new HashMap<>();
+
+ /** Whether this buffer pool has been destroyed or not. */
+ @GuardedBy("buffers")
+ private boolean destroyed;
+
+ /** Whether this buffer pool has been initialized or not. */
+ @GuardedBy("buffers")
+ private boolean initialized;
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize) {
+ // 5 min default buffer request timeout
+ this(totalBytes, bufferSize, 5 * 60 * 1000);
+ }
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize, long
requestTimeout) {
Review comment:
Let's annotate it as `@VisibleForTesting` and make it package private.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPool.java
##########
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A fixed-size {@link MemorySegment} pool used by batch shuffle for shuffle
data read (currently
+ * only used by sort-merge blocking shuffle).
+ */
+public class BatchShuffleReadBufferPool {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(BatchShuffleReadBufferPool.class);
+
+ /** Minimum total memory size in bytes of this buffer pool. */
+ public static final int MIN_TOTAL_BYTES = 32 * 1024 * 1024;
+
+ /**
+ * Memory size in bytes can be allocated from this buffer pool for a
single request (8M is for
+ * better sequential read).
+ */
+ public static final int NUM_BYTES_PER_REQUEST = 8 * 1024 * 1024;
+
+ /** Total direct memory size in bytes can can be allocated and used by
this buffer pool. */
+ private final long totalBytes;
+
+ /**
+ * Maximum time to wait in milliseconds when requesting read buffers from
this buffer pool
+ * before throwing an exception.
+ */
+ private final long requestTimeout;
+
+ /** The number of total buffers in this buffer pool. */
+ private final int numTotalBuffers;
+
+ /** Size of each buffer in bytes in this buffer pool. */
+ private final int bufferSize;
+
+ /** The number of buffers to be returned for a single request. */
+ private final int numBuffersPerRequest;
+
+ /** All available buffers in this buffer pool currently. */
+ @GuardedBy("buffers")
+ private final Queue<MemorySegment> buffers = new ArrayDeque<>();
+
+ /** Whether this buffer pool has been destroyed or not. */
+ @GuardedBy("buffers")
+ private boolean destroyed;
+
+ /** Whether this buffer pool has been initialized or not. */
+ @GuardedBy("buffers")
+ private boolean initialized;
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize) {
+ // 5 min default buffer request timeout
+ this(totalBytes, bufferSize, 5 * 60 * 1000);
+ }
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize, long
requestTimeout) {
+ checkArgument(totalBytes > 0, "Total memory size must be positive.");
+ checkArgument(bufferSize > 0, "Size of buffer must be positive.");
+ checkArgument(requestTimeout > 0, "Request timeout must be positive.");
+
+ this.totalBytes = totalBytes;
+ this.bufferSize = bufferSize;
+ this.requestTimeout = requestTimeout;
+
+ this.numTotalBuffers = (int) Math.min(totalBytes / bufferSize,
Integer.MAX_VALUE);
+ this.numBuffersPerRequest =
+ Math.min(numTotalBuffers, Math.max(1, NUM_BYTES_PER_REQUEST /
bufferSize));
+ }
+
+ @VisibleForTesting
+ public long getTotalBytes() {
+ return totalBytes;
+ }
+
+ @VisibleForTesting
Review comment:
Seems all these `VisibleForTesting` getters can be package private
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java
##########
@@ -180,7 +187,8 @@ public NettyShuffleEnvironment build() {
compressionCodec,
maxBuffersPerChannel,
sortShuffleMinBuffers,
- sortShuffleMinParallelism),
+ sortShuffleMinParallelism,
+ batchShuffleReadBuffers),
Review comment:
I guess it should be `batchShuffleReadBytes` and it should be placed in
ahead of the param `sortShuffleMinBuffers`.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * Copyright 2012 The Netty Project
+ * Copy from netty 4.1.32.Final
+ *
+ * The Netty Project licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.core.memory.MemorySegment;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for {@link BatchShuffleReadBufferPool}. */
+public class BatchShuffleReadBufferPoolTest {
+
+ @Rule public Timeout timeout = new Timeout(60, TimeUnit.SECONDS);
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalTotalBytes() {
+ createBufferPool(0, 1024);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalBufferSize() {
+ createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, 0);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalRequestTimeout() {
+ createBufferPool(0);
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testInitializeAfterDestroy() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ bufferPool.destroy();
+ bufferPool.initialize();
+ }
+
+ @Test
+ public void testLargeTotalBytes() {
+ BatchShuffleReadBufferPool bufferPool =
createBufferPool(Long.MAX_VALUE, 1024);
+ assertEquals(Integer.MAX_VALUE, bufferPool.getNumTotalBuffers());
+ bufferPool.destroy();
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testTotalBytesSmallerThanBufferSize() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool(4096, 32 *
1024);
+ try {
+ bufferPool.initialize();
+ } finally {
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testBufferCalculation() {
+ long totalBytes = BatchShuffleReadBufferPool.MIN_TOTAL_BYTES;
+ for (int bufferSize = 4 * 1024; bufferSize <= totalBytes; bufferSize
+= 1024) {
+ BatchShuffleReadBufferPool bufferPool =
+
createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, bufferSize);
+
+ assertEquals(totalBytes, bufferPool.getTotalBytes());
+ assertEquals(totalBytes / bufferSize,
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() <=
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() > 0);
+ }
+ }
+
+ @Test
+ public void testRequestBuffers() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ List<MemorySegment> buffers = new ArrayList<>();
+
+ try {
+ buffers.addAll(bufferPool.requestBuffers(this));
+ assertEquals(bufferPool.getNumBuffersPerRequest(), buffers.size());
+ } finally {
+ bufferPool.recycle(buffers, this);
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ bufferPool.destroy();
+ }
+ }
+
+ @Test(expected = TimeoutException.class)
+ public void testRequestBuffersTimeout() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool(1000);
+ List<MemorySegment> buffers = new ArrayList<>();
+
+ try {
+ for (int i = 0; i < 4; ++i) {
+ buffers.addAll(bufferPool.requestBuffers(this));
+ }
+ bufferPool.requestBuffers(new Object());
+ } finally {
+ bufferPool.recycle(buffers, this);
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testBufferFulfilledByRecycledBuffers() throws Exception {
+ int numRequestThreads = 2;
+ AtomicReference<Throwable> exception = new AtomicReference<>();
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ Map<Object, List<MemorySegment>> buffers = new ConcurrentHashMap<>();
+
+ try {
+ Object[] owners = new Object[] {new Object(), new Object(), new
Object(), new Object()};
+ for (int i = 0; i < 4; ++i) {
+ buffers.put(owners[i], bufferPool.requestBuffers(owners[i]));
+ }
+ assertEquals(0, bufferPool.getAvailableBuffers());
+
+ Thread[] requestThreads = new Thread[numRequestThreads];
+ for (int i = 0; i < numRequestThreads; ++i) {
+ requestThreads[i] =
+ new Thread(
+ () -> {
+ try {
+ Object owner = new Object();
+ buffers.put(owner,
bufferPool.requestBuffers(owner));
+ } catch (Throwable throwable) {
+ exception.set(throwable);
+ }
+ });
+ requestThreads[i].start();
+ }
+
+ // recycle one by one
+ for (MemorySegment segment : buffers.remove(owners[0])) {
+ bufferPool.recycle(segment, owners[0]);
+ }
+
+ // bulk recycle
+ bufferPool.recycle(buffers.remove(owners[1]), owners[1]);
+
+ for (Thread requestThread : requestThreads) {
+ requestThread.join();
+ }
+
+ assertNull(exception.get());
+ assertEquals(0, bufferPool.getAvailableBuffers());
+ assertEquals(4, buffers.size());
+ } finally {
+ for (Object owner : buffers.keySet()) {
+ bufferPool.recycle(buffers.remove(owner), owner);
+ }
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testMultipleThreadRequestAndRecycle() throws Exception {
+ int numRequestThreads = 10;
+ AtomicReference<Throwable> exception = new AtomicReference<>();
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+
+ try {
+ Thread[] requestThreads = new Thread[numRequestThreads];
+ for (int i = 0; i < numRequestThreads; ++i) {
+ requestThreads[i] =
+ new Thread(
+ () -> {
+ try {
+ Object owner = new Object();
+ for (int j = 0; j < 100; ++j) {
+ List<MemorySegment> buffers =
+
bufferPool.requestBuffers(owner);
+ Thread.sleep(10);
+ if (j % 2 == 0) {
+ bufferPool.recycle(buffers,
owner);
+ } else {
+ for (MemorySegment segment :
buffers) {
+
bufferPool.recycle(segment, owner);
+ }
+ }
+ }
+ } catch (Throwable throwable) {
+ exception.set(throwable);
+ }
+ });
+ requestThreads[i].start();
+ }
+
+ for (Thread requestThread : requestThreads) {
+ requestThread.join();
+ }
+
+ assertNull(exception.get());
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ } finally {
+ bufferPool.destroy();
+ }
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testDestroy() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ bufferPool.initialize();
+
+ assertFalse(bufferPool.isDestroyed());
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+
+ List<MemorySegment> buffers = bufferPool.requestBuffers(this);
+ assertEquals(
+ bufferPool.getNumTotalBuffers() - buffers.size(),
bufferPool.getAvailableBuffers());
+
+ bufferPool.destroy();
+ assertTrue(bufferPool.isDestroyed());
+ assertEquals(0, bufferPool.getAvailableBuffers());
+
+ bufferPool.recycle(buffers, this);
Review comment:
maybe add a separate test `testRecycleAfterPoolDestroyed` to test that
recycled buffers will be directly discarded if the pool is destroyed?
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * Copyright 2012 The Netty Project
+ * Copy from netty 4.1.32.Final
+ *
+ * The Netty Project licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.core.memory.MemorySegment;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for {@link BatchShuffleReadBufferPool}. */
+public class BatchShuffleReadBufferPoolTest {
+
+ @Rule public Timeout timeout = new Timeout(60, TimeUnit.SECONDS);
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalTotalBytes() {
+ createBufferPool(0, 1024);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalBufferSize() {
+ createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, 0);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalRequestTimeout() {
+ createBufferPool(0);
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testInitializeAfterDestroy() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ bufferPool.destroy();
+ bufferPool.initialize();
+ }
+
+ @Test
+ public void testLargeTotalBytes() {
+ BatchShuffleReadBufferPool bufferPool =
createBufferPool(Long.MAX_VALUE, 1024);
+ assertEquals(Integer.MAX_VALUE, bufferPool.getNumTotalBuffers());
+ bufferPool.destroy();
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testTotalBytesSmallerThanBufferSize() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool(4096, 32 *
1024);
+ try {
+ bufferPool.initialize();
+ } finally {
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testBufferCalculation() {
+ long totalBytes = BatchShuffleReadBufferPool.MIN_TOTAL_BYTES;
+ for (int bufferSize = 4 * 1024; bufferSize <= totalBytes; bufferSize
+= 1024) {
+ BatchShuffleReadBufferPool bufferPool =
+
createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, bufferSize);
+
+ assertEquals(totalBytes, bufferPool.getTotalBytes());
+ assertEquals(totalBytes / bufferSize,
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() <=
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() > 0);
+ }
+ }
+
+ @Test
+ public void testRequestBuffers() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ List<MemorySegment> buffers = new ArrayList<>();
+
+ try {
+ buffers.addAll(bufferPool.requestBuffers(this));
+ assertEquals(bufferPool.getNumBuffersPerRequest(), buffers.size());
+ } finally {
+ bufferPool.recycle(buffers, this);
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
Review comment:
this check looks irrelevant to this test case. Can we remove it and add
a separate test case `testRecycle` to test the normal simple cycle case,
because currently I can only see some complex recycle test cases?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A fixed-size {@link MemorySegment} pool used by batch shuffle for shuffle
data read (currently
+ * only used by sort-merge blocking shuffle).
+ */
+@Internal
+public class BatchShuffleReadBufferPool {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(BatchShuffleReadBufferPool.class);
+
+ /** Minimum total memory size in bytes of this buffer pool. */
+ public static final int MIN_TOTAL_BYTES = 32 * 1024 * 1024;
+
+ /**
+ * Memory size in bytes can be allocated from this buffer pool for a
single request (8M is for
+ * better sequential read).
+ */
+ public static final int NUM_BYTES_PER_REQUEST = 8 * 1024 * 1024;
+
+ /** Total direct memory size in bytes can can be allocated and used by
this buffer pool. */
+ private final long totalBytes;
+
+ /**
+ * Maximum time to wait in milliseconds when requesting read buffers from
this buffer pool
+ * before throwing an exception.
+ */
+ private final long requestTimeout;
+
+ /** The number of total buffers in this buffer pool. */
+ private final int numTotalBuffers;
+
+ /** Size of each buffer in bytes in this buffer pool. */
+ private final int bufferSize;
+
+ /** The number of buffers to be returned for a single request. */
+ private final int numBuffersPerRequest;
+
+ /**
+ * The maximum number of buffers can be allocated from this buffer pool
for a single buffer
+ * requester.
+ */
+ private final int maxBuffersPerRequester;
+
+ /** All available buffers in this buffer pool currently. */
+ @GuardedBy("buffers")
+ private final Queue<MemorySegment> buffers = new ArrayDeque<>();
+
+ /** Account for all the buffers requested per requester. */
+ @GuardedBy("buffers")
+ private final Map<Object, Counter> numBuffersAllocated = new HashMap<>();
+
+ /** Whether this buffer pool has been destroyed or not. */
+ @GuardedBy("buffers")
+ private boolean destroyed;
+
+ /** Whether this buffer pool has been initialized or not. */
+ @GuardedBy("buffers")
+ private boolean initialized;
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize) {
+ // 5 min default buffer request timeout
+ this(totalBytes, bufferSize, 5 * 60 * 1000);
+ }
+
+ public BatchShuffleReadBufferPool(long totalBytes, int bufferSize, long
requestTimeout) {
+ checkArgument(totalBytes > 0, "Total memory size must be positive.");
+ checkArgument(bufferSize > 0, "Size of buffer must be positive.");
+ checkArgument(requestTimeout > 0, "Request timeout must be positive.");
+
+ this.totalBytes = totalBytes;
+ this.bufferSize = bufferSize;
+ this.requestTimeout = requestTimeout;
+
+ this.numTotalBuffers = (int) Math.min(totalBytes / bufferSize,
Integer.MAX_VALUE);
+ this.numBuffersPerRequest =
+ Math.min(numTotalBuffers, Math.max(1, NUM_BYTES_PER_REQUEST /
bufferSize));
+ this.maxBuffersPerRequester =
+ Math.max(
+ 4 * numBuffersPerRequest,
+ numTotalBuffers /
BatchShuffleReadIOExecutor.MAX_NUM_THREADS);
+ }
+
+ public long getTotalBytes() {
+ return totalBytes;
+ }
+
+ public long getRequestTimeout() {
+ return requestTimeout;
+ }
+
+ public int getNumBuffersPerRequest() {
+ return numBuffersPerRequest;
+ }
+
+ public int getMaxBuffersPerRequester() {
+ return maxBuffersPerRequester;
+ }
+
+ public int getBufferSize() {
+ return bufferSize;
+ }
+
+ public int getNumTotalBuffers() {
+ return numTotalBuffers;
+ }
+
+ public int getAvailableBuffers() {
+ synchronized (buffers) {
+ return buffers.size();
+ }
+ }
+
+ /** Initializes this buffer pool which allocates all the buffers. */
+ public void initialize() {
+ LOG.info(
+ "Initializing batch shuffle IO buffer pool: numBuffers={},
bufferSize={}.",
+ numTotalBuffers,
+ bufferSize);
+
+ checkState(
+ totalBytes >= bufferSize,
+ String.format(
+ "Illegal configuration, config value for %s must be no
smaller than %s,"
+ + " please increase %s to at least %s bytes.",
+
TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key(),
+ TaskManagerOptions.MEMORY_SEGMENT_SIZE.key(),
+
TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key(),
+ bufferSize));
+
+ synchronized (buffers) {
+ checkState(!destroyed, "Buffer pool is already destroyed.");
+
+ if (initialized) {
+ return;
+ }
+ initialized = true;
+
+ try {
+ for (int i = 0; i < numTotalBuffers; ++i) {
+
buffers.add(MemorySegmentFactory.allocateUnpooledOffHeapMemory(bufferSize));
+ }
+ } catch (OutOfMemoryError outOfMemoryError) {
+ int allocated = buffers.size();
+ buffers.clear();
+ throw new OutOfMemoryError(
+ String.format(
+ "Can't allocate enough direct buffer for batch
shuffle read buffer "
+ + "pool (bytes allocated: %d, bytes
still needed: %d). To "
+ + "avoid the exception, you need to do
one of the following"
+ + " adjustments: 1) If you have ever
decreased %s, you need"
+ + " to undo the decrement; 2) If you
ever increased %s, you"
+ + " should also increase %s; 3) If
neither the above cases,"
+ + " it usually means some other parts
of your application "
+ + "have consumed too many direct
memory and the value of %s"
+ + " should be increased.",
+ allocated * bufferSize,
+ (numTotalBuffers - allocated) * bufferSize,
+
TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY.key(),
+
TaskManagerOptions.NETWORK_BATCH_SHUFFLE_READ_MEMORY.key(),
+
TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY.key(),
+
TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key()));
+ }
+ }
+ }
+
+ /**
+ * Requests a collection of buffers (determined by {@link
#numBuffersPerRequest}) from this
+ * buffer pool. Exception will be thrown if no enough buffers can be
allocated in the given
+ * timeout and the corresponding requester is not holding any allocated
buffers currently.
+ */
+ public List<MemorySegment> requestBuffers(Object owner) throws Exception {
+ checkArgument(owner != null, "Owner must be not null.");
+
+ List<MemorySegment> allocated = new ArrayList<>(numBuffersPerRequest);
+ synchronized (buffers) {
+ checkState(!destroyed, "Buffer pool is already destroyed.");
+
+ if (!initialized) {
+ initialize();
+ }
+
+ long startTime = System.currentTimeMillis();
+ Counter counter = numBuffersAllocated.get(owner);
+ while (buffers.size() < numBuffersPerRequest
+ || (counter != null
+ && counter.get() + numBuffersPerRequest >
maxBuffersPerRequester)) {
+ checkState(!destroyed, "Buffer pool is already destroyed.");
+
+ buffers.wait(requestTimeout);
Review comment:
close this comment because it is outdated with the removal of
`maxBuffersPerRequester`
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * Copyright 2012 The Netty Project
+ * Copy from netty 4.1.32.Final
+ *
+ * The Netty Project licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.core.memory.MemorySegment;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for {@link BatchShuffleReadBufferPool}. */
+public class BatchShuffleReadBufferPoolTest {
+
+ @Rule public Timeout timeout = new Timeout(60, TimeUnit.SECONDS);
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalTotalBytes() {
+ createBufferPool(0, 1024);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalBufferSize() {
+ createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, 0);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalRequestTimeout() {
+ createBufferPool(0);
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testInitializeAfterDestroy() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ bufferPool.destroy();
+ bufferPool.initialize();
+ }
+
+ @Test
+ public void testLargeTotalBytes() {
+ BatchShuffleReadBufferPool bufferPool =
createBufferPool(Long.MAX_VALUE, 1024);
+ assertEquals(Integer.MAX_VALUE, bufferPool.getNumTotalBuffers());
+ bufferPool.destroy();
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testTotalBytesSmallerThanBufferSize() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool(4096, 32 *
1024);
+ try {
+ bufferPool.initialize();
+ } finally {
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testBufferCalculation() {
+ long totalBytes = BatchShuffleReadBufferPool.MIN_TOTAL_BYTES;
+ for (int bufferSize = 4 * 1024; bufferSize <= totalBytes; bufferSize
+= 1024) {
+ BatchShuffleReadBufferPool bufferPool =
+
createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, bufferSize);
+
+ assertEquals(totalBytes, bufferPool.getTotalBytes());
+ assertEquals(totalBytes / bufferSize,
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() <=
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() > 0);
+ }
+ }
+
+ @Test
+ public void testRequestBuffers() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ List<MemorySegment> buffers = new ArrayList<>();
+
+ try {
+ buffers.addAll(bufferPool.requestBuffers(this));
+ assertEquals(bufferPool.getNumBuffersPerRequest(), buffers.size());
+ } finally {
+ bufferPool.recycle(buffers, this);
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ bufferPool.destroy();
+ }
+ }
+
+ @Test(expected = TimeoutException.class)
+ public void testRequestBuffersTimeout() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool(1000);
+ List<MemorySegment> buffers = new ArrayList<>();
+
+ try {
+ for (int i = 0; i < 4; ++i) {
+ buffers.addAll(bufferPool.requestBuffers(this));
+ }
+ bufferPool.requestBuffers(new Object());
+ } finally {
+ bufferPool.recycle(buffers, this);
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
Review comment:
same as the above comment. Let's have a separate test case for this
check.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * Copyright 2012 The Netty Project
+ * Copy from netty 4.1.32.Final
+ *
+ * The Netty Project licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.runtime.io.disk;
+
+import org.apache.flink.core.memory.MemorySegment;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for {@link BatchShuffleReadBufferPool}. */
+public class BatchShuffleReadBufferPoolTest {
+
+ @Rule public Timeout timeout = new Timeout(60, TimeUnit.SECONDS);
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalTotalBytes() {
+ createBufferPool(0, 1024);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalBufferSize() {
+ createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, 0);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testIllegalRequestTimeout() {
+ createBufferPool(0);
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testInitializeAfterDestroy() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ bufferPool.destroy();
+ bufferPool.initialize();
+ }
+
+ @Test
+ public void testLargeTotalBytes() {
+ BatchShuffleReadBufferPool bufferPool =
createBufferPool(Long.MAX_VALUE, 1024);
+ assertEquals(Integer.MAX_VALUE, bufferPool.getNumTotalBuffers());
+ bufferPool.destroy();
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testTotalBytesSmallerThanBufferSize() {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool(4096, 32 *
1024);
+ try {
+ bufferPool.initialize();
+ } finally {
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testBufferCalculation() {
+ long totalBytes = BatchShuffleReadBufferPool.MIN_TOTAL_BYTES;
+ for (int bufferSize = 4 * 1024; bufferSize <= totalBytes; bufferSize
+= 1024) {
+ BatchShuffleReadBufferPool bufferPool =
+
createBufferPool(BatchShuffleReadBufferPool.MIN_TOTAL_BYTES, bufferSize);
+
+ assertEquals(totalBytes, bufferPool.getTotalBytes());
+ assertEquals(totalBytes / bufferSize,
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() <=
bufferPool.getNumTotalBuffers());
+ assertTrue(bufferPool.getNumBuffersPerRequest() > 0);
+ }
+ }
+
+ @Test
+ public void testRequestBuffers() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ List<MemorySegment> buffers = new ArrayList<>();
+
+ try {
+ buffers.addAll(bufferPool.requestBuffers(this));
+ assertEquals(bufferPool.getNumBuffersPerRequest(), buffers.size());
+ } finally {
+ bufferPool.recycle(buffers, this);
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ bufferPool.destroy();
+ }
+ }
+
+ @Test(expected = TimeoutException.class)
+ public void testRequestBuffersTimeout() throws Exception {
+ BatchShuffleReadBufferPool bufferPool = createBufferPool(1000);
+ List<MemorySegment> buffers = new ArrayList<>();
+
+ try {
+ for (int i = 0; i < 4; ++i) {
+ buffers.addAll(bufferPool.requestBuffers(this));
+ }
+ bufferPool.requestBuffers(new Object());
+ } finally {
+ bufferPool.recycle(buffers, this);
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testBufferFulfilledByRecycledBuffers() throws Exception {
+ int numRequestThreads = 2;
+ AtomicReference<Throwable> exception = new AtomicReference<>();
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+ Map<Object, List<MemorySegment>> buffers = new ConcurrentHashMap<>();
+
+ try {
+ Object[] owners = new Object[] {new Object(), new Object(), new
Object(), new Object()};
+ for (int i = 0; i < 4; ++i) {
+ buffers.put(owners[i], bufferPool.requestBuffers(owners[i]));
+ }
+ assertEquals(0, bufferPool.getAvailableBuffers());
+
+ Thread[] requestThreads = new Thread[numRequestThreads];
+ for (int i = 0; i < numRequestThreads; ++i) {
+ requestThreads[i] =
+ new Thread(
+ () -> {
+ try {
+ Object owner = new Object();
+ buffers.put(owner,
bufferPool.requestBuffers(owner));
+ } catch (Throwable throwable) {
+ exception.set(throwable);
+ }
+ });
+ requestThreads[i].start();
+ }
+
+ // recycle one by one
+ for (MemorySegment segment : buffers.remove(owners[0])) {
+ bufferPool.recycle(segment, owners[0]);
+ }
+
+ // bulk recycle
+ bufferPool.recycle(buffers.remove(owners[1]), owners[1]);
+
+ for (Thread requestThread : requestThreads) {
+ requestThread.join();
+ }
+
+ assertNull(exception.get());
+ assertEquals(0, bufferPool.getAvailableBuffers());
+ assertEquals(4, buffers.size());
+ } finally {
+ for (Object owner : buffers.keySet()) {
+ bufferPool.recycle(buffers.remove(owner), owner);
+ }
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ bufferPool.destroy();
+ }
+ }
+
+ @Test
+ public void testMultipleThreadRequestAndRecycle() throws Exception {
+ int numRequestThreads = 10;
+ AtomicReference<Throwable> exception = new AtomicReference<>();
+ BatchShuffleReadBufferPool bufferPool = createBufferPool();
+
+ try {
+ Thread[] requestThreads = new Thread[numRequestThreads];
+ for (int i = 0; i < numRequestThreads; ++i) {
+ requestThreads[i] =
+ new Thread(
+ () -> {
+ try {
+ Object owner = new Object();
+ for (int j = 0; j < 100; ++j) {
+ List<MemorySegment> buffers =
+
bufferPool.requestBuffers(owner);
+ Thread.sleep(10);
+ if (j % 2 == 0) {
+ bufferPool.recycle(buffers,
owner);
+ } else {
+ for (MemorySegment segment :
buffers) {
+
bufferPool.recycle(segment, owner);
+ }
+ }
+ }
+ } catch (Throwable throwable) {
+ exception.set(throwable);
+ }
+ });
+ requestThreads[i].start();
+ }
+
+ for (Thread requestThread : requestThreads) {
+ requestThread.join();
+ }
+
+ assertNull(exception.get());
+ assertEquals(bufferPool.getNumTotalBuffers(),
bufferPool.getAvailableBuffers());
+ } finally {
+ bufferPool.destroy();
+ }
+ }
+
+ @Test(expected = IllegalStateException.class)
Review comment:
could we have this test case to test normal case only and have a
separate test case `testRequestBuffersAfterPoolDestroyed` to test
`requestBuffers` after destroy?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]