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:52:13 UTC

[flink] branch release-1.14 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 release-1.14
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.14 by this push:
     new e7df5ec  [FLINK-24846][streaming] Ignoring completing async operator record if mailbox is closed already
e7df5ec is described below

commit e7df5ec81fea6d072ea435f26ee338e938d3ea32
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;