This is an automated email from the ASF dual-hosted git repository.
ckj pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git
The following commit(s) were added to refs/heads/master by this push:
new 60762265 [ISSUE-509] Fix Flaky Test:
ShuffleBufferManagerTest#shuffleFlushThreshold (#510)
60762265 is described below
commit 60762265f8679ef26d8e2d2e736889014769f112
Author: xianjingfeng <[email protected]>
AuthorDate: Sun Jan 29 19:01:28 2023 +0800
[ISSUE-509] Fix Flaky Test: ShuffleBufferManagerTest#shuffleFlushThreshold
(#510)
### What changes were proposed in this pull request?
The reason is that the flush operation is asynchronous, the first block
maybe been flushed when judging the memory size.
So the solution is to support active trigger flush.
### Why are the changes needed?
Fix #509 Flaky Test: ShuffleBufferManagerTest#shuffleFlushThreshold.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
No need
Co-authored-by: Kaijie Chen <[email protected]>
---
.../apache/uniffle/server/ShuffleFlushManager.java | 84 +++++++++++++---------
.../uniffle/server/TestShuffleFlushManager.java | 46 ++++++++++++
.../server/buffer/ShuffleBufferManagerTest.java | 10 +--
3 files changed, 101 insertions(+), 39 deletions(-)
diff --git
a/server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java
b/server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java
index dc862195..baaf2955 100644
--- a/server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java
+++ b/server/src/main/java/org/apache/uniffle/server/ShuffleFlushManager.java
@@ -21,6 +21,7 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Executor;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
@@ -50,8 +51,8 @@ public class ShuffleFlushManager {
private static final Logger LOG =
LoggerFactory.getLogger(ShuffleFlushManager.class);
public static final AtomicLong ATOMIC_EVENT_ID = new AtomicLong(0);
private final ShuffleServer shuffleServer;
- private final BlockingQueue<ShuffleDataFlushEvent> flushQueue =
Queues.newLinkedBlockingQueue();
- private final ThreadPoolExecutor threadPoolExecutor;
+ protected final BlockingQueue<ShuffleDataFlushEvent> flushQueue =
Queues.newLinkedBlockingQueue();
+ private final Executor threadPoolExecutor;
private final List<String> storageBasePaths;
private final String shuffleServerId;
private final String storageType;
@@ -81,40 +82,10 @@ public class ShuffleFlushManager {
this.maxConcurrencyOfSingleOnePartition =
shuffleServerConf.get(ShuffleServerConf.SERVER_MAX_CONCURRENCY_OF_ONE_PARTITION);
- int waitQueueSize = shuffleServerConf.getInteger(
- ShuffleServerConf.SERVER_FLUSH_THREAD_POOL_QUEUE_SIZE);
- BlockingQueue<Runnable> waitQueue =
Queues.newLinkedBlockingQueue(waitQueueSize);
- int poolSize =
shuffleServerConf.getInteger(ShuffleServerConf.SERVER_FLUSH_THREAD_POOL_SIZE);
- long keepAliveTime =
shuffleServerConf.getLong(ShuffleServerConf.SERVER_FLUSH_THREAD_ALIVE);
- threadPoolExecutor = new ThreadPoolExecutor(poolSize, poolSize,
keepAliveTime, TimeUnit.SECONDS, waitQueue,
- ThreadUtils.getThreadFactory("FlushEventThreadPool"));
storageBasePaths =
shuffleServerConf.get(ShuffleServerConf.RSS_STORAGE_BASE_PATH);
pendingEventTimeoutSec =
shuffleServerConf.getLong(ShuffleServerConf.PENDING_EVENT_TIMEOUT_SEC);
- // the thread for flush data
- Runnable processEventRunnable = () -> {
- while (true) {
- try {
- ShuffleDataFlushEvent event = flushQueue.take();
- threadPoolExecutor.execute(() -> {
- try {
- ShuffleServerMetrics.gaugeWriteHandler.inc();
- flushToFile(event);
- } catch (Exception e) {
- LOG.error("Exception happened when flush data for " + event, e);
- } finally {
- ShuffleServerMetrics.gaugeWriteHandler.dec();
- ShuffleServerMetrics.gaugeEventQueueSize.dec();
- }
- });
- } catch (Exception e) {
- LOG.error("Exception happened when process event.", e);
- }
- }
- };
- Thread processEventThread = new Thread(processEventRunnable);
- processEventThread.setName("ProcessEventThread");
- processEventThread.setDaemon(true);
- processEventThread.start();
+ threadPoolExecutor = createFlushEventExecutor();
+ startEventProcessor();
// todo: extract a class named Service, and support stop method
Thread thread = new Thread("PendingEventProcessThread") {
@Override
@@ -137,6 +108,24 @@ public class ShuffleFlushManager {
thread.start();
}
+ private void startEventProcessor() {
+ // the thread for flush data
+ Thread processEventThread = new Thread(this::eventLoop);
+ processEventThread.setName("ProcessEventThread");
+ processEventThread.setDaemon(true);
+ processEventThread.start();
+ }
+
+ protected Executor createFlushEventExecutor() {
+ int waitQueueSize = shuffleServerConf.getInteger(
+ ShuffleServerConf.SERVER_FLUSH_THREAD_POOL_QUEUE_SIZE);
+ BlockingQueue<Runnable> waitQueue =
Queues.newLinkedBlockingQueue(waitQueueSize);
+ int poolSize =
shuffleServerConf.getInteger(ShuffleServerConf.SERVER_FLUSH_THREAD_POOL_SIZE);
+ long keepAliveTime =
shuffleServerConf.getLong(ShuffleServerConf.SERVER_FLUSH_THREAD_ALIVE);
+ return new ThreadPoolExecutor(poolSize, poolSize, keepAliveTime,
TimeUnit.SECONDS, waitQueue,
+ ThreadUtils.getThreadFactory("FlushEventThreadPool"));
+ }
+
public void addToFlushQueue(ShuffleDataFlushEvent event) {
if (!flushQueue.offer(event)) {
LOG.warn("Flush queue is full, discard event: " + event);
@@ -145,6 +134,33 @@ public class ShuffleFlushManager {
}
}
+ protected void eventLoop() {
+ while (true) {
+ processNextEvent();
+ }
+ }
+
+ protected void processNextEvent() {
+ try {
+ ShuffleDataFlushEvent event = flushQueue.take();
+ threadPoolExecutor.execute(() -> processEvent(event));
+ } catch (Exception e) {
+ LOG.error("Exception happened when process event.", e);
+ }
+ }
+
+ private void processEvent(ShuffleDataFlushEvent event) {
+ try {
+ ShuffleServerMetrics.gaugeWriteHandler.inc();
+ flushToFile(event);
+ } catch (Exception e) {
+ LOG.error("Exception happened when flush data for " + event, e);
+ } finally {
+ ShuffleServerMetrics.gaugeWriteHandler.dec();
+ ShuffleServerMetrics.gaugeEventQueueSize.dec();
+ }
+ }
+
private void flushToFile(ShuffleDataFlushEvent event) {
long start = System.currentTimeMillis();
boolean writeSuccess = false;
diff --git
a/server/src/test/java/org/apache/uniffle/server/TestShuffleFlushManager.java
b/server/src/test/java/org/apache/uniffle/server/TestShuffleFlushManager.java
new file mode 100644
index 00000000..675b765e
--- /dev/null
+++
b/server/src/test/java/org/apache/uniffle/server/TestShuffleFlushManager.java
@@ -0,0 +1,46 @@
+/*
+ * 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.uniffle.server;
+
+import java.util.concurrent.Executor;
+
+import org.apache.uniffle.server.storage.StorageManager;
+
+public class TestShuffleFlushManager extends ShuffleFlushManager {
+ public TestShuffleFlushManager(ShuffleServerConf shuffleServerConf, String
shuffleServerId,
+ ShuffleServer shuffleServer, StorageManager
storageManager) {
+ super(shuffleServerConf, shuffleServerId, shuffleServer, storageManager);
+ }
+
+ @Override
+ protected void eventLoop() {
+ // do nothing
+ }
+
+ @Override
+ protected Executor createFlushEventExecutor() {
+ return Runnable::run;
+ }
+
+ public void flush() {
+ while (!flushQueue.isEmpty()) {
+ processNextEvent();
+ }
+ }
+
+}
diff --git
a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferManagerTest.java
b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferManagerTest.java
index fcfe8306..e6952e7d 100644
---
a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferManagerTest.java
+++
b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferManagerTest.java
@@ -37,6 +37,7 @@ import org.apache.uniffle.server.ShuffleServerConf;
import org.apache.uniffle.server.ShuffleServerMetrics;
import org.apache.uniffle.server.ShuffleTaskManager;
import org.apache.uniffle.server.StatusCode;
+import org.apache.uniffle.server.TestShuffleFlushManager;
import org.apache.uniffle.server.storage.StorageManager;
import org.apache.uniffle.server.storage.StorageManagerFactory;
import org.apache.uniffle.storage.util.StorageType;
@@ -566,7 +567,7 @@ public class ShuffleBufferManagerTest extends
BufferTestBase {
serverConf.set(ShuffleServerConf.SERVER_SHUFFLE_FLUSH_THRESHOLD, 64L);
StorageManager storageManager =
StorageManagerFactory.getInstance().createStorageManager(conf);
- ShuffleFlushManager shuffleFlushManager = new ShuffleFlushManager(conf,
+ TestShuffleFlushManager shuffleFlushManager = new
TestShuffleFlushManager(conf,
"serverId", mockShuffleServer, storageManager);
shuffleBufferManager = new ShuffleBufferManager(serverConf,
shuffleFlushManager);
@@ -583,7 +584,7 @@ public class ShuffleBufferManagerTest extends
BufferTestBase {
assertEquals(96, shuffleBufferManager.getUsedMemory());
shuffleBufferManager.cacheShuffleData(appId, smallShuffleId, false,
createData(0, 31));
assertEquals(96 + 63, shuffleBufferManager.getUsedMemory());
- waitForFlush(shuffleFlushManager, appId, shuffleId, 1);
+ shuffleFlushManager.flush();
// small shuffle id is kept in memory
assertEquals(63, shuffleBufferManager.getUsedMemory());
assertEquals(0, shuffleBufferManager.getInFlushSize());
@@ -592,7 +593,7 @@ public class ShuffleBufferManagerTest extends
BufferTestBase {
shuffleBufferManager.cacheShuffleData(appId, smallShuffleId, false,
createData(0, 31));
shuffleBufferManager.cacheShuffleData(appId, smallShuffleId, false,
createData(0, 31));
assertEquals(63 * 3, shuffleBufferManager.getUsedMemory());
- waitForFlush(shuffleFlushManager, appId, smallShuffleId, 3);
+ shuffleFlushManager.flush();
assertEquals(0, shuffleBufferManager.getUsedMemory());
assertEquals(0, shuffleBufferManager.getInFlushSize());
@@ -601,8 +602,7 @@ public class ShuffleBufferManagerTest extends
BufferTestBase {
shuffleBufferManager.cacheShuffleData(appId, smallShuffleId, false,
createData(0, 21));
shuffleBufferManager.cacheShuffleData(appId, smallShuffleIdTwo, false,
createData(0, 20));
assertEquals(54 + 53 + 52, shuffleBufferManager.getUsedMemory());
- waitForFlush(shuffleFlushManager, appId, shuffleId, 2);
- waitForFlush(shuffleFlushManager, appId, smallShuffleId, 4);
+ shuffleFlushManager.flush();
assertEquals(52, shuffleBufferManager.getUsedMemory());
assertEquals(0, shuffleBufferManager.getInFlushSize());
}