You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by pn...@apache.org on 2021/12/15 12:51:26 UTC
[flink] branch master updated: [FLINK-24846][streaming] Ignoring completing async operator record if mailbox is closed already
This is an automated email from the ASF dual-hosted git repository.
pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 4065bfb [FLINK-24846][streaming] Ignoring completing async operator record if mailbox is closed already
4065bfb is described below
commit 4065bfba917d8ddb992f7d82290806ec6161ce10
Author: Anton Kalashnikov <ka...@yandex.ru>
AuthorDate: Tue Nov 30 15:21:26 2021 +0100
[FLINK-24846][streaming] Ignoring completing async operator record if mailbox is closed already
---
.../api/operators/async/AsyncWaitOperator.java | 15 +++++-
.../api/operators/async/AsyncWaitOperatorTest.java | 53 ++++++++++++++++++++++
2 files changed, 66 insertions(+), 2 deletions(-)
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
index dcd56ed..eeaf341 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
@@ -52,6 +52,7 @@ import javax.annotation.Nonnull;
import java.util.Collection;
import java.util.Collections;
import java.util.Optional;
+import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -301,8 +302,18 @@ public class AsyncWaitOperator<IN, OUT>
queue.emitCompletedElement(timestampedCollector);
// if there are more completed elements, emit them with subsequent mails
if (queue.hasCompletedElements()) {
- mailboxExecutor.execute(
- this::outputCompletedElement, "AsyncWaitOperator#outputCompletedElement");
+ try {
+ mailboxExecutor.execute(
+ this::outputCompletedElement,
+ "AsyncWaitOperator#outputCompletedElement");
+ } catch (RejectedExecutionException mailboxClosedException) {
+ // This exception can only happen if the operator is cancelled which means all
+ // pending records can be safely ignored since they will be processed one more
+ // time after recovery.
+ LOG.debug(
+ "Attempt to complete element is ignored since the mailbox rejected the execution.",
+ mailboxClosedException);
+ }
}
}
}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
index ea28bce..9916a89 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
@@ -59,6 +59,8 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarness;
import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarnessBuilder;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.testutils.junit.SharedReference;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.TestLogger;
@@ -68,6 +70,7 @@ import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
+import org.testcontainers.shaded.com.google.common.collect.Lists;
import java.util.ArrayDeque;
import java.util.ArrayList;
@@ -110,6 +113,7 @@ public class AsyncWaitOperatorTest extends TestLogger {
private static final long TIMEOUT = 1000L;
@Rule public Timeout timeoutRule = new Timeout(100, TimeUnit.SECONDS);
+ @Rule public final SharedObjects sharedObjects = SharedObjects.create();
private abstract static class MyAbstractAsyncFunction<IN>
extends RichAsyncFunction<IN, Integer> {
@@ -1022,6 +1026,55 @@ public class AsyncWaitOperatorTest extends TestLogger {
assertThat(outputElements, Matchers.equalTo(expectedOutput));
}
+ @Test
+ public void testIgnoreAsyncOperatorRecordsOnDrain() throws Exception {
+ // given: Async wait operator which are able to collect result futures.
+ StreamTaskMailboxTestHarnessBuilder<Integer> builder =
+ new StreamTaskMailboxTestHarnessBuilder<>(
+ OneInputStreamTask::new, BasicTypeInfo.INT_TYPE_INFO)
+ .addInput(BasicTypeInfo.INT_TYPE_INFO);
+ SharedReference<List<ResultFuture<?>>> resultFutures = sharedObjects.add(new ArrayList<>());
+ try (StreamTaskMailboxTestHarness<Integer> harness =
+ builder.setupOutputForSingletonOperatorChain(
+ new AsyncWaitOperatorFactory<>(
+ new CollectableFuturesAsyncFunction<>(resultFutures),
+ TIMEOUT,
+ 5,
+ AsyncDataStream.OutputMode.ORDERED))
+ .build()) {
+ // when: Processing at least two elements in reverse order to keep completed queue not
+ // empty.
+ harness.processElement(new StreamRecord<>(1));
+ harness.processElement(new StreamRecord<>(2));
+
+ for (ResultFuture<?> resultFuture : Lists.reverse(resultFutures.get())) {
+ resultFuture.complete(Collections.emptyList());
+ }
+
+ // then: All records from async operator should be ignored during drain since they will
+ // be processed on recovery.
+ harness.finishProcessing();
+ assertTrue(harness.getOutput().isEmpty());
+ }
+ }
+
+ private static class CollectableFuturesAsyncFunction<IN> implements AsyncFunction<IN, IN> {
+
+ private static final long serialVersionUID = -4214078239227288637L;
+
+ private final SharedReference<List<ResultFuture<?>>> resultFutures;
+
+ private CollectableFuturesAsyncFunction(
+ SharedReference<List<ResultFuture<?>>> resultFutures) {
+ this.resultFutures = resultFutures;
+ }
+
+ @Override
+ public void asyncInvoke(IN input, ResultFuture<IN> resultFuture) throws Exception {
+ resultFutures.get().add(resultFuture);
+ }
+ }
+
private static class ControllableAsyncFunction<IN> implements AsyncFunction<IN, IN> {
private static final long serialVersionUID = -4214078239267288636L;