You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/14 12:03:21 UTC

[GitHub] [flink] yunfengzhou-hub opened a new pull request, #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

yunfengzhou-hub opened a new pull request, #20275:
URL: https://github.com/apache/flink/pull/20275

   ## What is the purpose of the change
   
   Now, the operator events sent from a coordinator to its subtasks would be temporarily blocked when the coordinator starts checkpointing and be unblocked after the checkpoint barriers are injected into the sources. This PR extends the duration of this blocking period so that an event for a subtask is blocked until the subtask completes the current checkpoint. By doing this, this PR preserves the consistency of these operator events during checkpoints and failovers.
   
   This PR is a component of FLINK-26029, which aims to generalize the checkpoint protocol of operator coordinators and makes coordinators applicable for non-source operators as well.
   
   ## Brief change log
   
   - The subtask gateway can be closed/reopened and replaces the operator event valve.
   - An operator event type is introduced to notify the coordinator about a subtask completing a checkpoint.
   - Subtasks would send the operator event above to their coordinators after snapshotting their states.
   - The holder of an operator coordinator reopens a subtask gateway after it receives the operator event above, instead of after checkpoint barriers are injected into sources.
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
   - Added unit tests to verify the correctness of the closing/reopening behavior of subtask gateways.
   - Added integration tests to verify that the operator events sent from coordinators to subtask would not be lost after failover, even if the events are sent during checkpoints.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (yes)
       - This PR additionally blocks the communication between a coordinator and its subtasks from when checkpoint barriers are sent to sources to when subtasks complete the checkpoint. This means that an operator event generated during checkpoints would be delivered only after the target subtask completes the checkpoint, bringing longer latency to that event.
   
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes)
       - This PR affects the latency of operator events from coordinators to subtasks, as described above.
   
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926212561


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorEventDispatcherImpl.java:
##########
@@ -82,6 +83,10 @@ public void registerEventHandler(OperatorID operator, OperatorEventHandler handl
         }
     }
 
+    Set<OperatorID> getRegisteredOperators() {

Review Comment:
   As already discussed in the comment about `openGatewayAndUnmarkCheckpoint` above, the default package-private decorator is suitable for these methods.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r928493949


##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java:
##########
@@ -59,15 +66,25 @@ public static CompletableFuture<CoordinatorSnapshot> triggerCoordinatorCheckpoin
 
     public static CompletableFuture<AllCoordinatorSnapshots> triggerAllCoordinatorCheckpoints(
             final Collection<OperatorCoordinatorCheckpointContext> coordinators,
-            final long checkpointId)
+            final PendingCheckpoint checkpoint)
             throws Exception {
 
         final Collection<CompletableFuture<CoordinatorSnapshot>> individualSnapshots =
                 new ArrayList<>(coordinators.size());
 
         for (final OperatorCoordinatorCheckpointContext coordinator : coordinators) {
+            Set<Integer> subtasksToCheckpoint = new HashSet<>();

Review Comment:
   I agree. I'll rename it to `subtasksToWaitFor`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1191298692

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r931717326


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -229,6 +233,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
+        Preconditions.checkState(!context.isConcurrentExecutionAttemptsSupported());

Review Comment:
   I think this check is necessary. The correctness of the current implementation of this PR is based on the assumption that speculative execution will not function together with Flink's checkpoint mechanism, thus it is reasonable and critical to check that there will not be multiple execution attempts when a checkpoint starts. 
   
   The newly added checks in other functions seem removable. I'll make modifications to those invocations.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r936161829


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {

Review Comment:
   Can we make the test name consistent with each other?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testFailingUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!shouldCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!sentEventAfterCkpt) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator} that has the
+     * following additional behavior:
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure would be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the next.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinator
+            extends CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator {
+        private static final long NO_CHECKPOINT = -1;
+
+        /** The coordinator should have completed checkpoints before sending out this message. */
+        private final int checkpointBeforeMessage;
+
+        /**
+         * The id of a completed checkpoint for the job.
+         *
+         * <ul>
+         *   <li>If not set(equal to {@link #NO_CHECKPOINT}), it will be set to the next completed
+         *       checkpoint.
+         *   <li>If the coordinator is restored from the checkpoint, it will be set to the restored
+         *       checkpoint.
+         *   <li>Otherwise, its value will not be changed.
+         * </ul>
+         *
+         * <p>It is used to assist the setting of {@link #hasCompletedCheckpointForCoordinator} and
+         * {@link #hasCompletedCheckpointForJob}.
+         */
+        private long markedCompletedCheckpointId;

Review Comment:
   How about `lastCompletedCheckpointId`?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/CoordinatorEventsExactlyOnceITCase.java:
##########
@@ -350,10 +345,10 @@ public void start() throws Exception {}
         @Override
         public void close() throws Exception {
             scheduledExecutor.shutdownNow();
-            assertTrue(scheduledExecutor.awaitTermination(10, TimeUnit.MINUTES));
+            assertThat(scheduledExecutor.awaitTermination(10, TimeUnit.MINUTES)).isTrue();
 
             mailboxExecutor.shutdownNow();
-            assertTrue(mailboxExecutor.awaitTermination(10, TimeUnit.MINUTES));
+            assertThat(scheduledExecutor.awaitTermination(10, TimeUnit.MINUTES)).isTrue();

Review Comment:
   Do you mean to use `mailboxExecutor` here?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testFailingUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!shouldCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!sentEventAfterCkpt) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator} that has the
+     * following additional behavior:
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure would be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the next.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinator
+            extends CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator {
+        private static final long NO_CHECKPOINT = -1;
+
+        /** The coordinator should have completed checkpoints before sending out this message. */
+        private final int checkpointBeforeMessage;
+
+        /**
+         * The id of a completed checkpoint for the job.
+         *
+         * <ul>
+         *   <li>If not set(equal to {@link #NO_CHECKPOINT}), it will be set to the next completed
+         *       checkpoint.
+         *   <li>If the coordinator is restored from the checkpoint, it will be set to the restored
+         *       checkpoint.
+         *   <li>Otherwise, its value will not be changed.
+         * </ul>
+         *
+         * <p>It is used to assist the setting of {@link #hasCompletedCheckpointForCoordinator} and
+         * {@link #hasCompletedCheckpointForJob}.
+         */
+        private long markedCompletedCheckpointId;
+
+        /** Whether the coordinator has completed any checkpoint. */
+        private boolean hasCompletedCheckpointForCoordinator;
+
+        /** Whether the whole job (both coordinator and operator) has completed any checkpoint. */
+        private boolean hasCompletedCheckpointForJob;
+
+        public EventSendingCoordinator(Context context, String name, int numEvents, int delay) {
+            super(context, name, numEvents, delay);
+            this.checkpointBeforeMessage = new Random().nextInt(numEvents / 4);
+            this.markedCompletedCheckpointId = NO_CHECKPOINT;
+            this.hasCompletedCheckpointForCoordinator = false;
+            this.hasCompletedCheckpointForJob = false;
+        }
+
+        @Override
+        protected void sendNextEvent() {
+            if (!hasCompletedCheckpointForCoordinator && nextNumber >= checkpointBeforeMessage) {
+                return;
+            }
+
+            if (!hasCompletedCheckpointForJob && nextNumber >= failAtMessage) {
+                return;
+            }
+
+            super.sendNextEvent();
+
+            if (!sentEventAfterCkpt && hasCompletedCheckpointForCoordinator) {
+                sentEventAfterCkpt = true;
+            }
+        }
+
+        @Override
+        protected void handleCheckpoint() {
+            if (nextToComplete != null && sentEventAfterCkpt && !testScript.hasAlreadyFailed()) {
+                testScript.recordHasFailed();
+                context.failJob(new Exception("test failure"));
+            }
+
+            if (nextToComplete != null) {
+                hasCompletedCheckpointForCoordinator = true;
+            }
+
+            super.handleCheckpoint();
+        }
+
+        @Override
+        public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData)
+                throws Exception {
+            super.resetToCheckpoint(checkpointId, checkpointData);
+            markedCompletedCheckpointId = checkpointId;
+            hasCompletedCheckpointForCoordinator = true;
+            hasCompletedCheckpointForJob = true;
+        }
+
+        @Override
+        public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> result)
+                throws Exception {
+            super.checkpointCoordinator(checkpointId, result);
+            if (markedCompletedCheckpointId == NO_CHECKPOINT) {
+                markedCompletedCheckpointId = checkpointId;

Review Comment:
   I suppose a checkpoint is only completed after `notifyCheckpointComplete()` is invoked. Is it correct to update it here?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;

Review Comment:
   How about `isEventSentAfterFirstCheckpoint`?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testFailingUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!shouldCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!sentEventAfterCkpt) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator} that has the
+     * following additional behavior:
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure would be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the next.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinator
+            extends CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator {
+        private static final long NO_CHECKPOINT = -1;
+
+        /** The coordinator should have completed checkpoints before sending out this message. */
+        private final int checkpointBeforeMessage;

Review Comment:
   Would it be better to re-name it as `numMessagesBeforeFirstCheckpoint`?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testFailingUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!shouldCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!sentEventAfterCkpt) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator} that has the
+     * following additional behavior:
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure would be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the next.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinator
+            extends CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator {
+        private static final long NO_CHECKPOINT = -1;
+
+        /** The coordinator should have completed checkpoints before sending out this message. */
+        private final int checkpointBeforeMessage;
+
+        /**
+         * The id of a completed checkpoint for the job.
+         *
+         * <ul>
+         *   <li>If not set(equal to {@link #NO_CHECKPOINT}), it will be set to the next completed
+         *       checkpoint.
+         *   <li>If the coordinator is restored from the checkpoint, it will be set to the restored
+         *       checkpoint.
+         *   <li>Otherwise, its value will not be changed.
+         * </ul>
+         *
+         * <p>It is used to assist the setting of {@link #hasCompletedCheckpointForCoordinator} and
+         * {@link #hasCompletedCheckpointForJob}.
+         */
+        private long markedCompletedCheckpointId;
+
+        /** Whether the coordinator has completed any checkpoint. */
+        private boolean hasCompletedCheckpointForCoordinator;
+
+        /** Whether the whole job (both coordinator and operator) has completed any checkpoint. */
+        private boolean hasCompletedCheckpointForJob;

Review Comment:
   Is it possible to remove this variable and use `markedCompletedCheckpointId == NO_CHECKPOINT` instead?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testFailingUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!shouldCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!sentEventAfterCkpt) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator} that has the
+     * following additional behavior:
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure would be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the next.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinator
+            extends CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator {
+        private static final long NO_CHECKPOINT = -1;
+
+        /** The coordinator should have completed checkpoints before sending out this message. */
+        private final int checkpointBeforeMessage;
+
+        /**
+         * The id of a completed checkpoint for the job.
+         *
+         * <ul>
+         *   <li>If not set(equal to {@link #NO_CHECKPOINT}), it will be set to the next completed
+         *       checkpoint.
+         *   <li>If the coordinator is restored from the checkpoint, it will be set to the restored
+         *       checkpoint.
+         *   <li>Otherwise, its value will not be changed.
+         * </ul>
+         *
+         * <p>It is used to assist the setting of {@link #hasCompletedCheckpointForCoordinator} and
+         * {@link #hasCompletedCheckpointForJob}.
+         */
+        private long markedCompletedCheckpointId;
+
+        /** Whether the coordinator has completed any checkpoint. */
+        private boolean hasCompletedCheckpointForCoordinator;
+
+        /** Whether the whole job (both coordinator and operator) has completed any checkpoint. */
+        private boolean hasCompletedCheckpointForJob;
+
+        public EventSendingCoordinator(Context context, String name, int numEvents, int delay) {
+            super(context, name, numEvents, delay);
+            this.checkpointBeforeMessage = new Random().nextInt(numEvents / 4);
+            this.markedCompletedCheckpointId = NO_CHECKPOINT;
+            this.hasCompletedCheckpointForCoordinator = false;
+            this.hasCompletedCheckpointForJob = false;
+        }
+
+        @Override
+        protected void sendNextEvent() {
+            if (!hasCompletedCheckpointForCoordinator && nextNumber >= checkpointBeforeMessage) {
+                return;
+            }
+
+            if (!hasCompletedCheckpointForJob && nextNumber >= failAtMessage) {
+                return;
+            }
+
+            super.sendNextEvent();
+
+            if (!sentEventAfterCkpt && hasCompletedCheckpointForCoordinator) {
+                sentEventAfterCkpt = true;
+            }
+        }
+
+        @Override
+        protected void handleCheckpoint() {
+            if (nextToComplete != null && sentEventAfterCkpt && !testScript.hasAlreadyFailed()) {
+                testScript.recordHasFailed();
+                context.failJob(new Exception("test failure"));
+            }
+
+            if (nextToComplete != null) {
+                hasCompletedCheckpointForCoordinator = true;
+            }
+
+            super.handleCheckpoint();
+        }
+
+        @Override
+        public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData)
+                throws Exception {
+            super.resetToCheckpoint(checkpointId, checkpointData);
+            markedCompletedCheckpointId = checkpointId;
+            hasCompletedCheckpointForCoordinator = true;
+            hasCompletedCheckpointForJob = true;
+        }
+
+        @Override
+        public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> result)
+                throws Exception {
+            super.checkpointCoordinator(checkpointId, result);
+            if (markedCompletedCheckpointId == NO_CHECKPOINT) {

Review Comment:
   Should it be run in the mailbox thread?
   
   Same for other methods that access coordinator states.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testFailingUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!shouldCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!sentEventAfterCkpt) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator} that has the
+     * following additional behavior:
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure would be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the next.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinator
+            extends CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator {
+        private static final long NO_CHECKPOINT = -1;
+
+        /** The coordinator should have completed checkpoints before sending out this message. */
+        private final int checkpointBeforeMessage;
+
+        /**
+         * The id of a completed checkpoint for the job.
+         *
+         * <ul>
+         *   <li>If not set(equal to {@link #NO_CHECKPOINT}), it will be set to the next completed
+         *       checkpoint.
+         *   <li>If the coordinator is restored from the checkpoint, it will be set to the restored
+         *       checkpoint.
+         *   <li>Otherwise, its value will not be changed.
+         * </ul>
+         *
+         * <p>It is used to assist the setting of {@link #hasCompletedCheckpointForCoordinator} and
+         * {@link #hasCompletedCheckpointForJob}.
+         */
+        private long markedCompletedCheckpointId;
+
+        /** Whether the coordinator has completed any checkpoint. */
+        private boolean hasCompletedCheckpointForCoordinator;

Review Comment:
   How about `isCoordinatorFirstCheckpointCompleted`?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 1;
+
+    /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+    private static boolean sentEventAfterCkpt;
+
+    /** Whether the {@link IdlingSourceFunction} should be closed to finish the job. */
+    private static boolean shouldCloseSource;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource = false;
+        sentEventAfterCkpt = false;
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testFailingUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new FailingCheckpointOperatorFactory<>("failingCheckpoint", NUM_EVENTS, DELAY));
+        assertThat(TestScript.getForOperator("failingCheckpoint-subtask0").hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!shouldCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!sentEventAfterCkpt) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator} that has the
+     * following additional behavior:
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure would be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the next.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinator
+            extends CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator {
+        private static final long NO_CHECKPOINT = -1;
+
+        /** The coordinator should have completed checkpoints before sending out this message. */
+        private final int checkpointBeforeMessage;
+
+        /**
+         * The id of a completed checkpoint for the job.
+         *
+         * <ul>
+         *   <li>If not set(equal to {@link #NO_CHECKPOINT}), it will be set to the next completed
+         *       checkpoint.
+         *   <li>If the coordinator is restored from the checkpoint, it will be set to the restored
+         *       checkpoint.
+         *   <li>Otherwise, its value will not be changed.

Review Comment:
   Should this doc focus on explaining the semantics of this variable? If so, it seems that we can just explain it refers to the ID of the last completed checkpoint. What does it mean to say `will not be changed`?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r931933271


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators.
+ */
+public class CoordinatorEventsExactlyOnceTest {
+    @ClassRule
+    public static final MiniClusterResource MINI_CLUSTER =
+            new MiniClusterResource(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setNumberSlotsPerTaskManager(2)
+                            .build());
+
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void test() throws Exception {
+        long count =
+                executeAndGetNumReceivedEvents(
+                        env,
+                        new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        long count =
+                executeAndGetNumReceivedEvents(
+                        env,
+                        new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {

Review Comment:
   According to our offline discussion, I'll add custom exactly-once guarantees in the failing operator and verify the correctness of the output results of this test.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r934612070


##########
flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/CoordinatorEventsExactlyOnceITCase.java:
##########
@@ -111,30 +108,21 @@
  * <h2>2. Exactly-once alignment between multiple Coordinators</h2>
  *
  * <p>After a coordinator completed its checkpoint future, all events sent after that must be held
- * back until the checkpoint barriers have been sent to the sources. That is because from the
- * coordinator's perspective, the events are after the checkpoint, so they must also be after the
- * checkpoint from the source task's perspective.
+ * back until its subtasks completed their checkpoint. That is because from the coordinator's
+ * perspective, the events are after the checkpoint, so they must also be after the checkpoint from
+ * the subtask's perspective.
  *
  * <p>When multiple coordinators exist, there are time spans during which some coordinators finished
  * their checkpoints, but others did not yet, and hence the source checkpoint barriers are not yet
  * injected (that happens only once all coordinators are done with their checkpoint). The events
- * from the earlier coordinators must be blocked until all coordinators finished their checkpoints
- * and the source checkpoint barriers are injected.
- *
- * <p>In the example below, the events {@code c & d} must be held back until after the barrier
- * injection.
- *
- * <pre>
- * Coordinator one events: => a . . b . |trigger| . . |complete| . . c . . d . |barrier| . e . f
- * Coordinator two events: => . . x . . |trigger| . . . . . . . . . .|complete||barrier| . . y . . z
- * </pre>
+ * from the earlier coordinators must be blocked until all coordinators finished their checkpoints ,

Review Comment:
   `checkpoints ,` -> `checkpoints,`
   
   And `finished` seems to be inconsistent with `complete` in the next line.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new FailingCheckpointOperatorFactory<>("failingCheckpoint", numEvents, delay));
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, OneInputStreamOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // Differs the parallelism of source operators from that of tested operators, so that when
+        // checkpoint barriers are injected into sources, the tested operators have not started
+        // checkpoint yet.
+        DataStream<Long> stream =
+                env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                        .setParallelism(2);
+        stream =
+                stream.transform("eventReceiving", TypeInformation.of(Long.class), factory)
+                        .setParallelism(1);
+        stream.addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        long count = executionResult.getAccumulatorResult(EventReceivingOperator.COUNTER_NAME);
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        private boolean isCancelled = false;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCancelled && !shouldCloseSource.get()) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {
+            isCancelled = true;
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+        private final String name;
+        protected final int numEvents;

Review Comment:
   Can you improve the readability/consistency of the code here?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new FailingCheckpointOperatorFactory<>("failingCheckpoint", numEvents, delay));
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, OneInputStreamOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // Differs the parallelism of source operators from that of tested operators, so that when
+        // checkpoint barriers are injected into sources, the tested operators have not started
+        // checkpoint yet.
+        DataStream<Long> stream =
+                env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                        .setParallelism(2);
+        stream =
+                stream.transform("eventReceiving", TypeInformation.of(Long.class), factory)
+                        .setParallelism(1);
+        stream.addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        long count = executionResult.getAccumulatorResult(EventReceivingOperator.COUNTER_NAME);
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        private boolean isCancelled = false;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCancelled && !shouldCloseSource.get()) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {
+            isCancelled = true;
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+        private final String name;
+        protected final int numEvents;
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * The stream operator that receives the events and accumulates the numbers. The task is
+     * stateful and checkpoints the accumulator.
+     */
+    private static class EventReceivingOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T>, OperatorEventHandler {
+        protected static final String COUNTER_NAME = "numEvents";
+
+        protected final LongCounter counter = new LongCounter();
+
+        protected ListState<Long> state;
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+            getRuntimeContext().addAccumulator(COUNTER_NAME, counter);
+        }
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                shouldCloseSource.set(true);
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            state.update(Collections.singletonList(counter.getLocalValue()));
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+
+            state =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "counterState", BasicTypeInfo.LONG_TYPE_INFO));
+
+            counter.resetLocal();
+            Iterator<Long> iterator = state.get().iterator();
+            if (iterator.hasNext()) {
+                counter.add(iterator.next());
+            }
+            Preconditions.checkArgument(!iterator.hasNext());
+
+            // signal the coordinator to start
+            getContainingTask()
+                    .getEnvironment()
+                    .getOperatorCoordinatorEventGateway()
+                    .sendOperatorEventToCoordinator(
+                            getOperatorID(),
+                            new SerializedValue<>(
+                                    new StartEvent(counter.getLocalValue().intValue() - 1)));
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link FailingCheckpointOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class FailingCheckpointOperatorFactory<IN, OUT>
+            extends EventReceivingOperatorFactory<IN, OUT> {
+        public FailingCheckpointOperatorFactory(String name, int numEvents, int delay) {
+            super(name, numEvents, delay);
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new FailingCheckpointOperator<>(numEvents);
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+    }
+
+    /**
+     * A subclass of {@link EventReceivingOperator} whose subtask would fail during a specific
+     * checkpoint.
+     */
+    private static class FailingCheckpointOperator<T> extends EventReceivingOperator<T> {
+
+        private final int failAtCheckpointAfterMessage;
+
+        private TestScript testScript;
+
+        private FailingCheckpointOperator(int numEvents) {
+            this.failAtCheckpointAfterMessage =
+                    numEvents * 2 / 3 + new Random().nextInt(numEvents / 6);
+        }
+
+        @Override
+        public void setup(
+                StreamTask<?, ?> containingTask,
+                StreamConfig config,
+                Output<StreamRecord<T>> output) {
+            super.setup(containingTask, config, output);
+            if (containingTask.getIndexInSubtaskGroup() == 0) {
+                this.testScript = TestScript.getForOperator(getOperatorID() + "-subtask0");
+            } else {
+                this.testScript = null;
+            }
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                if (testScript.hasAlreadyFailed()) {

Review Comment:
   Is it possible that `testScript == null` here? 



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new FailingCheckpointOperatorFactory<>("failingCheckpoint", numEvents, delay));
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, OneInputStreamOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // Differs the parallelism of source operators from that of tested operators, so that when
+        // checkpoint barriers are injected into sources, the tested operators have not started
+        // checkpoint yet.
+        DataStream<Long> stream =
+                env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                        .setParallelism(2);
+        stream =
+                stream.transform("eventReceiving", TypeInformation.of(Long.class), factory)
+                        .setParallelism(1);
+        stream.addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        long count = executionResult.getAccumulatorResult(EventReceivingOperator.COUNTER_NAME);
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        private boolean isCancelled = false;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCancelled && !shouldCloseSource.get()) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {
+            isCancelled = true;
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+        private final String name;
+        protected final int numEvents;
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * The stream operator that receives the events and accumulates the numbers. The task is
+     * stateful and checkpoints the accumulator.
+     */
+    private static class EventReceivingOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T>, OperatorEventHandler {
+        protected static final String COUNTER_NAME = "numEvents";
+
+        protected final LongCounter counter = new LongCounter();
+
+        protected ListState<Long> state;
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+            getRuntimeContext().addAccumulator(COUNTER_NAME, counter);
+        }
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                shouldCloseSource.set(true);
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            state.update(Collections.singletonList(counter.getLocalValue()));
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+
+            state =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "counterState", BasicTypeInfo.LONG_TYPE_INFO));
+
+            counter.resetLocal();
+            Iterator<Long> iterator = state.get().iterator();
+            if (iterator.hasNext()) {
+                counter.add(iterator.next());
+            }
+            Preconditions.checkArgument(!iterator.hasNext());
+
+            // signal the coordinator to start
+            getContainingTask()
+                    .getEnvironment()
+                    .getOperatorCoordinatorEventGateway()
+                    .sendOperatorEventToCoordinator(
+                            getOperatorID(),
+                            new SerializedValue<>(
+                                    new StartEvent(counter.getLocalValue().intValue() - 1)));
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link FailingCheckpointOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class FailingCheckpointOperatorFactory<IN, OUT>
+            extends EventReceivingOperatorFactory<IN, OUT> {
+        public FailingCheckpointOperatorFactory(String name, int numEvents, int delay) {
+            super(name, numEvents, delay);
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new FailingCheckpointOperator<>(numEvents);
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+    }
+
+    /**
+     * A subclass of {@link EventReceivingOperator} whose subtask would fail during a specific
+     * checkpoint.
+     */
+    private static class FailingCheckpointOperator<T> extends EventReceivingOperator<T> {
+
+        private final int failAtCheckpointAfterMessage;
+
+        private TestScript testScript;
+
+        private FailingCheckpointOperator(int numEvents) {
+            this.failAtCheckpointAfterMessage =
+                    numEvents * 2 / 3 + new Random().nextInt(numEvents / 6);
+        }
+
+        @Override
+        public void setup(
+                StreamTask<?, ?> containingTask,
+                StreamConfig config,
+                Output<StreamRecord<T>> output) {
+            super.setup(containingTask, config, output);
+            if (containingTask.getIndexInSubtaskGroup() == 0) {
+                this.testScript = TestScript.getForOperator(getOperatorID() + "-subtask0");
+            } else {
+                this.testScript = null;
+            }
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                if (testScript.hasAlreadyFailed()) {
+                    shouldCloseSource.set(true);
+                }
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            if (counter.getLocalValue() > failAtCheckpointAfterMessage
+                    && testScript != null
+                    && !testScript.hasAlreadyFailed()) {
+                testScript.recordHasFailed();
+                throw new RuntimeException();

Review Comment:
   Have you verified that each invocation of `testFailingCheckpoint()` always triggers this exception before this test ends? Note that we typically don't want to have flaky tests that fail with low probability.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;

Review Comment:
   Is this variable name consistent with other `private static final` variable names in the repo?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1202474655

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926211546


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {
+            return;
+        }
+        openGatewayAndUnmarkCheckpoint();
+    }
+
+    /** Opens the gateway, releasing all buffered events. */
+    void openGatewayAndUnmarkCheckpoint() {
+        checkRunsInMainThread();
+
+        currentCheckpointId = NO_CHECKPOINT;
+        if (!isClosed) {
+            return;
+        }
+
+        for (BlockedEvent blockedEvent : blockedEvents) {
+            CompletableFuture<Acknowledge> result =
+                    innerGateway.sendEvent(blockedEvent.operatorEvent);
+            FutureUtils.forward(result, blockedEvent.future);
+        }
+        blockedEvents.clear();
+
+        isClosed = false;
+    }
+
+    @Override
+    public CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt) {
+        if (isClosed) {
+            CompletableFuture<Acknowledge> sendResult = new CompletableFuture<>();

Review Comment:
   You are right. I'll change the behavior here to always throw the exceptions if any.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926211762


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -277,29 +300,33 @@ private void checkpointCoordinatorInternal(
             final long checkpointId, final CompletableFuture<byte[]> result) {
         mainThreadExecutor.assertRunningInMainThread();
 
-        final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
-
-        FutureUtils.assertNoException(
-                coordinatorCheckpoint.handleAsync(
-                        (success, failure) -> {
-                            if (failure != null) {
-                                result.completeExceptionally(failure);
-                            } else if (eventValve.tryShutValve(checkpointId)) {
-                                completeCheckpointOnceEventsAreDone(checkpointId, result, success);
-                            } else {
-                                // if we cannot shut the valve, this means the checkpoint
-                                // has been aborted before, so the future is already
-                                // completed exceptionally. but we try to complete it here
-                                // again, just in case, as a safety net.
-                                result.completeExceptionally(
-                                        new FlinkException("Cannot shut event valve"));
-                            }
-                            return null;
-                        },
-                        mainThreadExecutor));
-
         try {
-            eventValve.markForCheckpoint(checkpointId);
+            subtaskGatewayMap.values().forEach(x -> x.markForCheckpoint(checkpointId));
+
+            final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
+
+            coordinatorCheckpoint.whenComplete(
+                    (success, failure) -> {
+                        if (failure != null) {
+                            result.completeExceptionally(failure);
+                        } else {
+                            closeGateways(checkpointId, result);

Review Comment:
   I agree. I'll reuse the `handleAsync()` as it originally did.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1209316439

   The latest code of this PR has been tested against the CI([link](https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=39709&view=results)). There is a test failure in this CI, `PulsarSourceITCase.testSavepoint`, which have existed before this PR on the master branch, for example in `e47cb6ee` of the master branch. The problem has been recorded in FLINK-26721.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926407667


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -245,7 +262,13 @@ public void notifyCheckpointAborted(long checkpointId) {
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
-        mainThreadExecutor.execute(() -> coordinator.notifyCheckpointAborted(checkpointId));
+        mainThreadExecutor.execute(
+                () -> {
+                    subtaskGatewayMap
+                            .values()
+                            .forEach(x -> x.openGatewayAndUnmarkCheckpoint(checkpointId));
+                    coordinator.notifyCheckpointAborted(checkpointId);

Review Comment:
   As we are removing the `afterSourceBarrierInjection()` and the invocation of `openGatewayAndUnmarkCheckpoint()` in it, we need to add this method to the possible afterward methods, including `handleEventFromOperator` and `notifyCheckpointAborted`.
   
   During our offline discussion, we doubted whether the invocation of this method in `abortCurrentTriggering` would be enough. I found that `notifyCheckpointAborted` and `abortCurrentTriggering` might be invoked independently, so it is still necessary to invoke this method in `notifyCheckpointAborted`. I'll add test cases in `CoordinatorEventsExactlyOnceTest` to verify this situation.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r927315690


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -284,22 +311,24 @@ private void checkpointCoordinatorInternal(
                         (success, failure) -> {
                             if (failure != null) {
                                 result.completeExceptionally(failure);
-                            } else if (eventValve.tryShutValve(checkpointId)) {
+                            } else if (closeGateways(checkpointId, subtasksToCheckpoint)) {
                                 completeCheckpointOnceEventsAreDone(checkpointId, result, success);
                             } else {
-                                // if we cannot shut the valve, this means the checkpoint
+                                // if we cannot close the gateway, this means the checkpoint
                                 // has been aborted before, so the future is already
                                 // completed exceptionally. but we try to complete it here
                                 // again, just in case, as a safety net.
                                 result.completeExceptionally(
-                                        new FlinkException("Cannot shut event valve"));
+                                        new FlinkException("Cannot close gateway"));
                             }
                             return null;
                         },
                         mainThreadExecutor));
 
         try {
-            eventValve.markForCheckpoint(checkpointId);
+            for (int subtask : subtasksToCheckpoint) {
+                subtaskGatewayMap.get(subtask).markForCheckpoint(checkpointId);

Review Comment:
   Would it be simpler to call `markForCheckpoint()` for every subtask instead of additionally passing `subtasksToCheckpoint` as a function input parameter?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -120,7 +118,7 @@
     private final OperatorID operatorId;
     private final LazyInitializedCoordinatorContext context;
     private final SubtaskAccess.SubtaskAccessFactory taskAccesses;
-    private final OperatorEventValve eventValve;
+    private final Map<Integer, SubtaskGatewayImpl> subtaskGatewayMap = new HashMap<>();

Review Comment:
   Should we instantiate this variable in the constructor for consistency with `unconfirmedEvents`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java:
##########
@@ -59,15 +66,25 @@ public static CompletableFuture<CoordinatorSnapshot> triggerCoordinatorCheckpoin
 
     public static CompletableFuture<AllCoordinatorSnapshots> triggerAllCoordinatorCheckpoints(
             final Collection<OperatorCoordinatorCheckpointContext> coordinators,
-            final long checkpointId)
+            final PendingCheckpoint checkpoint)
             throws Exception {
 
         final Collection<CompletableFuture<CoordinatorSnapshot>> individualSnapshots =
                 new ArrayList<>(coordinators.size());
 
         for (final OperatorCoordinatorCheckpointContext coordinator : coordinators) {
+            Set<Integer> subtasksToCheckpoint = new HashSet<>();

Review Comment:
   Is there any conceptual difference between `tasksToWaitFor` and `subtasksToCheckpoint`? If no, it is probably more readable to use a name that is consistent with `CheckpointPlan::getTasksToWaitFor()`.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -18,43 +18,82 @@
 
 package org.apache.flink.runtime.operators.coordination;
 
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.operators.coordination.util.IncompleteFuturesTracker;
 import org.apache.flink.runtime.util.Runnables;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.FutureUtils;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
 
 /**
  * Implementation of the {@link OperatorCoordinator.SubtaskGateway} interface that access to
  * subtasks for status and event sending via {@link SubtaskAccess}.
+ *
+ * <p>Instances of this class can be temporarily closed, blocking events from going through,
+ * buffering them, and releasing them later. It is used for "alignment" of operator event streams
+ * with checkpoint barrier injection, similar to how the input channels are aligned during a common
+ * checkpoint.
+ *
+ * <p>The methods on the critical communication path, including closing/reopening the gateway and
+ * sending the operator events, are required to be used in a single-threaded context specified by
+ * the {@link #mainThreadExecutor} in {@link #SubtaskGatewayImpl(SubtaskAccess,
+ * ComponentMainThreadExecutor, IncompleteFuturesTracker)} in order to avoid concurrent issues.

Review Comment:
   `concurrent issues` -> `race condition`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -308,6 +337,24 @@ private void checkpointCoordinatorInternal(
         }
     }
 
+    private boolean closeGateways(
+            final long checkpointId, final Set<Integer> subtasksToCheckpoint) {
+        boolean hasCloseableGateway = false;
+        for (int subtask : subtasksToCheckpoint) {
+            SubtaskGatewayImpl gateway = subtaskGatewayMap.get(subtask);
+            if (!gateway.tryCloseGateway(checkpointId)) {

Review Comment:
   Would it be simpler to call `tryCloseGateway()` for every subtask instead of only for `subtasksToCheckpoint`?
   
   The logic here seems to suggest that it is possible to have `tryCloseGateway(..)` fail for all tasks consistently. But it is not possible to have it fail for a subset of tasks. Can you explain this possibility and the impossibility?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventDispatcher.java:
##########
@@ -29,6 +29,9 @@ public interface OperatorEventDispatcher {
     /**
      * Register a listener that is notified every time an OperatorEvent is sent from the
      * OperatorCoordinator (of the operator with the given OperatorID) to this subtask.
+     *
+     * <p>The stream operator with the given OperatorID must implement {@link OperatorEventHandler}

Review Comment:
   The existing comment (prior to this PR) seems to have described the behavior of calling this method. Could you explain what is the extra benefit of adding this comment?
   
   And why does the operator with the given operatorID have to implement `OperatorEventHandler`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -18,43 +18,82 @@
 
 package org.apache.flink.runtime.operators.coordination;
 
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.operators.coordination.util.IncompleteFuturesTracker;
 import org.apache.flink.runtime.util.Runnables;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.FutureUtils;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
 
 /**
  * Implementation of the {@link OperatorCoordinator.SubtaskGateway} interface that access to
  * subtasks for status and event sending via {@link SubtaskAccess}.
+ *
+ * <p>Instances of this class can be temporarily closed, blocking events from going through,
+ * buffering them, and releasing them later. It is used for "alignment" of operator event streams
+ * with checkpoint barrier injection, similar to how the input channels are aligned during a common
+ * checkpoint.
+ *
+ * <p>The methods on the critical communication path, including closing/reopening the gateway and
+ * sending the operator events, are required to be used in a single-threaded context specified by
+ * the {@link #mainThreadExecutor} in {@link #SubtaskGatewayImpl(SubtaskAccess,
+ * ComponentMainThreadExecutor, IncompleteFuturesTracker)} in order to avoid concurrent issues.
  */
 class SubtaskGatewayImpl implements OperatorCoordinator.SubtaskGateway {
 
     private static final String EVENT_LOSS_ERROR_MESSAGE =
             "An OperatorEvent from an OperatorCoordinator to a task was lost. "
                     + "Triggering task failover to ensure consistency. Event: '%s', targetTask: %s";
 
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
     private final SubtaskAccess subtaskAccess;
-    private final EventSender sender;
-    private final Executor sendingExecutor;
+
+    private final ComponentMainThreadExecutor mainThreadExecutor;
+
     private final IncompleteFuturesTracker incompleteFuturesTracker;
 
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();

Review Comment:
   Should we initialize this variable in the constructor for consistency with e.g. `lastCheckpointId`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -18,43 +18,82 @@
 
 package org.apache.flink.runtime.operators.coordination;
 
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.operators.coordination.util.IncompleteFuturesTracker;
 import org.apache.flink.runtime.util.Runnables;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.FutureUtils;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
 
 /**
  * Implementation of the {@link OperatorCoordinator.SubtaskGateway} interface that access to
  * subtasks for status and event sending via {@link SubtaskAccess}.
+ *
+ * <p>Instances of this class can be temporarily closed, blocking events from going through,
+ * buffering them, and releasing them later. It is used for "alignment" of operator event streams
+ * with checkpoint barrier injection, similar to how the input channels are aligned during a common
+ * checkpoint.
+ *
+ * <p>The methods on the critical communication path, including closing/reopening the gateway and
+ * sending the operator events, are required to be used in a single-threaded context specified by
+ * the {@link #mainThreadExecutor} in {@link #SubtaskGatewayImpl(SubtaskAccess,
+ * ComponentMainThreadExecutor, IncompleteFuturesTracker)} in order to avoid concurrent issues.
  */
 class SubtaskGatewayImpl implements OperatorCoordinator.SubtaskGateway {
 
     private static final String EVENT_LOSS_ERROR_MESSAGE =
             "An OperatorEvent from an OperatorCoordinator to a task was lost. "
                     + "Triggering task failover to ensure consistency. Event: '%s', targetTask: %s";
 
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
     private final SubtaskAccess subtaskAccess;
-    private final EventSender sender;
-    private final Executor sendingExecutor;
+
+    private final ComponentMainThreadExecutor mainThreadExecutor;
+
     private final IncompleteFuturesTracker incompleteFuturesTracker;
 
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;

Review Comment:
   Should we explicitly initialize this variable?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -91,27 +130,133 @@ public CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt) {
                                                         new FlinkException(msg, failure)));
                             }
                         },
-                        sendingExecutor);
+                        mainThreadExecutor);
 
-        sendingExecutor.execute(
+        mainThreadExecutor.execute(
                 () -> {
-                    sender.sendEvent(sendAction, sendResult);
+                    sendEventInternal(sendAction, sendResult);
                     incompleteFuturesTracker.trackFutureWhileIncomplete(result);
                 });
+
         return result;
     }
 
-    @Override
-    public ExecutionAttemptID getExecution() {
-        return subtaskAccess.currentAttempt();
+    private void sendEventInternal(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        checkRunsInMainThread();
+
+        if (isClosed) {
+            blockedEvents.add(new BlockedEvent(sendAction, result));
+        } else {
+            callSendAction(sendAction, result);
+        }
     }
 
-    @Override
-    public int getSubtask() {
-        return subtaskAccess.getSubtaskIndex();
+    private void callSendAction(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        try {
+            final CompletableFuture<Acknowledge> sendResult = sendAction.call();
+            FutureUtils.forward(sendResult, result);
+        } catch (Throwable t) {
+            ExceptionUtils.rethrowIfFatalError(t);
+            result.completeExceptionally(t);
+        }
     }
 
-    private boolean isReady() {
-        return subtaskAccess.hasSwitchedToRunning().isDone();
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        checkRunsInMainThread();
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {

Review Comment:
   Prior to this PR, `OperatorEventVale::openValveAndUnmarkCheckpoint(long expectedCheckpointId)` throws IllegalStateException in this case.
   
   Could it be useful to still throw IllegalStateException if `currentCheckpointId != expectedCheckpointId && currentCheckpointId != NO_CHECKPOINT`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -18,43 +18,82 @@
 
 package org.apache.flink.runtime.operators.coordination;
 
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.operators.coordination.util.IncompleteFuturesTracker;
 import org.apache.flink.runtime.util.Runnables;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.FutureUtils;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
 
 /**
  * Implementation of the {@link OperatorCoordinator.SubtaskGateway} interface that access to
  * subtasks for status and event sending via {@link SubtaskAccess}.
+ *
+ * <p>Instances of this class can be temporarily closed, blocking events from going through,
+ * buffering them, and releasing them later. It is used for "alignment" of operator event streams
+ * with checkpoint barrier injection, similar to how the input channels are aligned during a common
+ * checkpoint.
+ *
+ * <p>The methods on the critical communication path, including closing/reopening the gateway and
+ * sending the operator events, are required to be used in a single-threaded context specified by
+ * the {@link #mainThreadExecutor} in {@link #SubtaskGatewayImpl(SubtaskAccess,
+ * ComponentMainThreadExecutor, IncompleteFuturesTracker)} in order to avoid concurrent issues.
  */
 class SubtaskGatewayImpl implements OperatorCoordinator.SubtaskGateway {
 
     private static final String EVENT_LOSS_ERROR_MESSAGE =
             "An OperatorEvent from an OperatorCoordinator to a task was lost. "
                     + "Triggering task failover to ensure consistency. Event: '%s', targetTask: %s";
 
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
     private final SubtaskAccess subtaskAccess;
-    private final EventSender sender;
-    private final Executor sendingExecutor;
+
+    private final ComponentMainThreadExecutor mainThreadExecutor;
+
     private final IncompleteFuturesTracker incompleteFuturesTracker;
 
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
     SubtaskGatewayImpl(
             SubtaskAccess subtaskAccess,
-            EventSender sender,
-            Executor sendingExecutor,
+            ComponentMainThreadExecutor mainThreadExecutor,
             IncompleteFuturesTracker incompleteFuturesTracker) {
         this.subtaskAccess = subtaskAccess;
-        this.sender = sender;
-        this.sendingExecutor = sendingExecutor;
+        this.mainThreadExecutor = mainThreadExecutor;
         this.incompleteFuturesTracker = incompleteFuturesTracker;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    @Override
+    public ExecutionAttemptID getExecution() {

Review Comment:
   nits: it is in general preferred not to move code around unless there is clear benefits (e.g. a code style pattern that we can consistently enforce in the future).
   
   Do we need to move these 3 methods here?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r930593185


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -308,6 +337,24 @@ private void checkpointCoordinatorInternal(
         }
     }
 
+    private boolean closeGateways(
+            final long checkpointId, final Set<Integer> subtasksToCheckpoint) {
+        boolean hasCloseableGateway = false;
+        for (int subtask : subtasksToCheckpoint) {
+            SubtaskGatewayImpl gateway = subtaskGatewayMap.get(subtask);
+            if (!gateway.tryCloseGateway(checkpointId)) {

Review Comment:
   If we do want to capture that bug, how about we consistently call `tryCloseGateway()` for every subtasks and check the condition at the end of the loop, instead of stopping at the first subtask whose `tryCloseGateway()` return false?
   
   Otherwise, suppose `subtask_1.tryCloseGateway() == false` and `subtask_2.tryCloseGateway() == true`, the method will not throw Exception. This makes the method's behavior harder to understand.
   
   
   



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/AcknowledgeCheckpointEvent.java:
##########
@@ -18,20 +18,16 @@
 
 package org.apache.flink.runtime.operators.coordination;
 
-import org.apache.flink.runtime.messages.Acknowledge;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.CompletableFuture;
+/**
+ * An {@link OperatorEvent} sent from a subtask to its {@link OperatorCoordinator} to signal that
+ * the checkpoint of an individual task is completed.
+ */
+public class AcknowledgeCheckpointEvent implements OperatorEvent {
 
-/** Simple interface for a component that takes and sends events. */
-@FunctionalInterface
-interface EventSender {
+    /** The ID of the checkpoint that this event is related to. */
+    final long checkpointId;

Review Comment:
   Is there any existing example where we make a variable protected and do not provide accessor method for this variable?
   
   If no, it seems better to keep code style consistent with existing code.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators.
+ */
+public class CoordinatorEventsExactlyOnceTest {
+    @ClassRule
+    public static final MiniClusterResource MINI_CLUSTER =
+            new MiniClusterResource(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setNumberSlotsPerTaskManager(2)
+                            .build());
+
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void test() throws Exception {

Review Comment:
   Give this test a more meaningful name?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators.
+ */
+public class CoordinatorEventsExactlyOnceTest {

Review Comment:
   Could you explain why we add a new class here instead of adding tests in `CoordinatorEventsExactlyOnceITCase`?
   
   If we do need to add new class, would you update class names and doc for clarify the difference?
   
   In general we want the test class name to self-explain its purpose. It is not clear what is the difference between `CoordinatorEventsExactlyOnceITCase` and `CoordinatorEventsExactlyOnceTest`, and which class should be used for new tests in the future.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -229,6 +233,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
+        Preconditions.checkState(!context.isConcurrentExecutionAttemptsSupported());

Review Comment:
   We typically only check condition when the program would produce incorrect result if this condition is not met. Otherwise it is easy to get program unnecessary inefficient over time.
   
   Could you double check that we do need to check *this condition* at *this place*?
   
   Same for other `Preconditions.checkState` added in this PR.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators.
+ */
+public class CoordinatorEventsExactlyOnceTest {
+    @ClassRule
+    public static final MiniClusterResource MINI_CLUSTER =
+            new MiniClusterResource(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setNumberSlotsPerTaskManager(2)
+                            .build());
+
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void test() throws Exception {
+        long count =
+                executeAndGetNumReceivedEvents(
+                        env,
+                        new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        long count =
+                executeAndGetNumReceivedEvents(
+                        env,
+                        new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {

Review Comment:
   What condition does this test check?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1200660902

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926210804


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {

Review Comment:
   As raised in a comment below, I agree that the closing/reopening behavior is needed by all subtask gateway implementations, so I'll remove this class and merge its functions to `SubtaskGatewayImpl`. In that case, this comment would not be applicable.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926210901


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -236,7 +240,20 @@ public void notifyCheckpointComplete(long checkpointId) {
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
-        mainThreadExecutor.execute(() -> coordinator.notifyCheckpointComplete(checkpointId));
+        mainThreadExecutor.execute(
+                () -> {
+                    coordinator.notifyCheckpointComplete(checkpointId);
+                    for (Map.Entry<Integer, CloseableSubtaskGateway> entry :
+                            subtaskGatewayMap.entrySet()) {
+                        if (entry.getValue().isClosed()) {
+                            LOG.warn(

Review Comment:
   I agree. I'll make the change.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926407667


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -245,7 +262,13 @@ public void notifyCheckpointAborted(long checkpointId) {
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
-        mainThreadExecutor.execute(() -> coordinator.notifyCheckpointAborted(checkpointId));
+        mainThreadExecutor.execute(
+                () -> {
+                    subtaskGatewayMap
+                            .values()
+                            .forEach(x -> x.openGatewayAndUnmarkCheckpoint(checkpointId));
+                    coordinator.notifyCheckpointAborted(checkpointId);

Review Comment:
   As we are removing the `afterSourceBarrierInjection()` and the invocation of `openGatewayAndUnmarkCheckpoint()` in it, we need to add this method to the possible afterward methods, including `handleEventFromOperator` and `notifyCheckpointAborted`.
   
   During our offline discussion, we doubted whether the invocation of this method in `abortCurrentTriggering` would be enough. I found that `notifyCheckpointAborted` and `abortCurrentTriggering` might be invoked independently, so it is still necessary to invoke this method in `notifyCheckpointAborted`. I have added test cases in `CoordinatorEventsExactlyOnceTest` to verify this situation.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r937256425


##########
flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/CoordinatorEventsExactlyOnceITCase.java:
##########
@@ -378,6 +373,10 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven
                         checkState(!workLoopRunning);
                         checkState(subtaskGateway != null);
 
+                        if (((StartEvent) event).lastValue >= 0) {
+                            nextNumber = ((StartEvent) event).lastValue + 1;

Review Comment:
   Suppose the job starts cleanly instead of from a global failover, it seems that `nextNumber` not explicitly initialized before it is first used, right?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 10;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        ManuallyClosedSourceFunction.isCloseSource = false;
+        EventSendingCoordinatorWithGuaranteedCheckpoint.isEventSentAfterFirstCheckpoint = false;
+    }
+
+    @Test
+    public void testCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testCheckpointWithSubtaskFailure() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testUnalignedCheckpointWithSubtaskFailure() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new ManuallyClosedSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and closes on demand. */
+    private static class ManuallyClosedSourceFunction<T> implements SourceFunction<T> {
+
+        /** Whether the source function should be closed to finish the job. */
+        private static boolean isCloseSource;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the first checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();

Review Comment:
   I am wondering why `BlockCheckpointBarrierOperator` does not block data flow from being forwarded the next operator.
   
   Should we do `output.collect(element)`?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 10;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        ManuallyClosedSourceFunction.isCloseSource = false;
+        EventSendingCoordinatorWithGuaranteedCheckpoint.isEventSentAfterFirstCheckpoint = false;
+    }
+
+    @Test
+    public void testCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testCheckpointWithSubtaskFailure() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testUnalignedCheckpointWithSubtaskFailure() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new ManuallyClosedSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and closes on demand. */
+    private static class ManuallyClosedSourceFunction<T> implements SourceFunction<T> {
+
+        /** Whether the source function should be closed to finish the job. */
+        private static boolean isCloseSource;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the first checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            while (!EventSendingCoordinatorWithGuaranteedCheckpoint
+                    .isEventSentAfterFirstCheckpoint) {
+                Thread.sleep(100);
+            }
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventSendingCoordinatorWithGuaranteedCheckpoint} and
+     * {@link EventReceivingOperator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+
+        protected final String name;
+
+        protected final int numEvents;
+
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinatorWithGuaranteedCheckpoint(
+                            context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * A subclass of {@link EventSendingCoordinator} that additionally guarantees the following
+     * behavior around checkpoint.
+     *
+     * <ul>
+     *   <li>The job must have completed a checkpoint before the coordinator injects the failure.
+     *   <li>The failure must be injected after the coordinator has completed its first checkpoint
+     *       and before it completes the second.
+     *   <li>There must be events being sent when the coordinator has completed the first checkpoint
+     *       while the subtask has not.
+     * </ul>
+     */
+    private static class EventSendingCoordinatorWithGuaranteedCheckpoint
+            extends EventSendingCoordinator {
+
+        /** Whether the coordinator has sent any event to its subtask after any checkpoint. */
+        private static boolean isEventSentAfterFirstCheckpoint;
+
+        /**
+         * The max number of elements the coordinator might send out before it completes the first
+         * checkpoint.
+         */
+        private final int numEventsBeforeFirstCheckpoint;
+
+        /** Whether the coordinator has completed the first checkpoint. */
+        private boolean isCoordinatorFirstCheckpointCompleted;
+
+        /** Whether the job (both coordinator and operator) has completed the first checkpoint. */
+        private boolean isJobFirstCheckpointCompleted;
+
+        public EventSendingCoordinatorWithGuaranteedCheckpoint(
+                Context context, String name, int numEvents, int delay) {
+            super(context, name, numEvents, delay);
+            this.numEventsBeforeFirstCheckpoint = new Random().nextInt(numEvents / 6);
+            this.isCoordinatorFirstCheckpointCompleted = false;
+            this.isJobFirstCheckpointCompleted = false;
+        }
+
+        @Override
+        protected void sendNextEvent() {
+            if (!isCoordinatorFirstCheckpointCompleted
+                    && nextNumber >= numEventsBeforeFirstCheckpoint) {
+                return;
+            }
+
+            if (!isJobFirstCheckpointCompleted && nextNumber >= numEventsBeforeFailure) {

Review Comment:
   I suppose the purpose of this check is trying to ensure that the test will ensure the first job checkpoint is completed before coordinator injects the failure, as explained in the java doc of `EventSendingCoordinatorWithGuaranteedCheckpoint`.
   
   However, it seems that this check allows the `nextNumber` to reach `numEventsBeforeFailure`, which actually allows `checkWhetherToTriggerFailure()` in the parent class to trigger failure before the first job checkpoint is completed.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 10;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        ManuallyClosedSourceFunction.isCloseSource = false;
+        EventSendingCoordinatorWithGuaranteedCheckpoint.isEventSentAfterFirstCheckpoint = false;
+    }
+
+    @Test
+    public void testCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testCheckpointWithSubtaskFailure() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testUnalignedCheckpointWithSubtaskFailure() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new ManuallyClosedSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and closes on demand. */
+    private static class ManuallyClosedSourceFunction<T> implements SourceFunction<T> {
+
+        /** Whether the source function should be closed to finish the job. */
+        private static boolean isCloseSource;

Review Comment:
   nits: it seems more intuitive to say `shouldCloseSource` or `isEndEventReceived`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1201951778

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r931932142


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -229,6 +233,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
+        Preconditions.checkState(!context.isConcurrentExecutionAttemptsSupported());

Review Comment:
   According to our offline discussion, I'll remove this check as well. `subtaskGatewayMap` would only be used without concurrent executions enabled, and it still manages one gateway for each subtask.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r932818606


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -121,7 +118,15 @@
     private final OperatorID operatorId;
     private final LazyInitializedCoordinatorContext context;
     private final SubtaskAccess.SubtaskAccessFactory taskAccesses;
-    private final OperatorEventValve eventValve;
+
+    /**
+     * A map that manages subtask gateways. It is used to control the opening/closing of each
+     * gateway during checkpoint. As checkpoints only work when concurrent execution attempt is

Review Comment:
   The following statement might be more explicit:
   
   `As checkpoints only work when concurrent execution attempt is ...` -> `This map should only be read when concurrent execution attempt is disabled. Note that concurrent execution attempt is currently guaranteed to be disabled when checkpoint is enabled.`



##########
flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java:
##########
@@ -228,4 +234,44 @@ public void triggerTaskFailover(Throwable cause) {
             // ignore this in the tests
         }
     }
+
+    private static class NoMainThreadCheckComponentMainThreadExecutor

Review Comment:
   Can you add comments for this class similar to how other `ComponentMainThreadExecutor` subclasses are documented?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -91,27 +130,133 @@ public CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt) {
                                                         new FlinkException(msg, failure)));
                             }
                         },
-                        sendingExecutor);
+                        mainThreadExecutor);
 
-        sendingExecutor.execute(
+        mainThreadExecutor.execute(
                 () -> {
-                    sender.sendEvent(sendAction, sendResult);
+                    sendEventInternal(sendAction, sendResult);
                     incompleteFuturesTracker.trackFutureWhileIncomplete(result);
                 });
+
         return result;
     }
 
-    @Override
-    public ExecutionAttemptID getExecution() {
-        return subtaskAccess.currentAttempt();
+    private void sendEventInternal(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        checkRunsInMainThread();
+
+        if (isClosed) {
+            blockedEvents.add(new BlockedEvent(sendAction, result));
+        } else {
+            callSendAction(sendAction, result);
+        }
     }
 
-    @Override
-    public int getSubtask() {
-        return subtaskAccess.getSubtaskIndex();
+    private void callSendAction(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        try {
+            final CompletableFuture<Acknowledge> sendResult = sendAction.call();
+            FutureUtils.forward(sendResult, result);
+        } catch (Throwable t) {
+            ExceptionUtils.rethrowIfFatalError(t);
+            result.completeExceptionally(t);
+        }
     }
 
-    private boolean isReady() {
-        return subtaskAccess.hasSwitchedToRunning().isDone();
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        checkRunsInMainThread();
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {

Review Comment:
   Suppose `currentCheckpointId = NO_CHECKPOINT` and `checkpointId = 10`, should we still throw exception in this case? If yes, then we need to update the condition accordingly.
   
   Also, since the reason for this condition is that not intuitive, could you add comments explaining this check? 
   
   



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint.
+ *
+ * <p>This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators and verifying the correctness of the behavior of stream operators.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new FailingCheckpointOperatorFactory<>("failingCheckpoint", numEvents, delay));
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, OneInputStreamOperatorFactory<Long, Long> factory)
+            throws Exception {
+        DataStream<Long> stream =
+                env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                        .setParallelism(2);
+        stream =
+                stream.transform("eventReceiving", TypeInformation.of(Long.class), factory)
+                        .setParallelism(1);
+        stream.addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        long count = executionResult.getAccumulatorResult(EventReceivingOperator.COUNTER_NAME);
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        private boolean isCancelled = false;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCancelled && !shouldCloseSource.get()) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {
+            isCancelled = true;
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+        private final String name;
+        protected final int numEvents;
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * The stream operator that receives the events and accumulates the numbers. The task is
+     * stateful and checkpoints the accumulator.
+     */
+    private static class EventReceivingOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T>, OperatorEventHandler {
+        protected static final String COUNTER_NAME = "numEvents";
+
+        protected final LongCounter counter = new LongCounter();
+
+        protected ListState<Long> state;
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+            getRuntimeContext().addAccumulator(COUNTER_NAME, counter);
+        }
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                shouldCloseSource.set(true);
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            state.update(Collections.singletonList(counter.getLocalValue()));
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+
+            state =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "counterState", BasicTypeInfo.LONG_TYPE_INFO));
+
+            counter.resetLocal();
+            Iterator<Long> iterator = state.get().iterator();
+            if (iterator.hasNext()) {
+                counter.add(iterator.next());
+            }
+            Preconditions.checkArgument(!iterator.hasNext());
+
+            // signal the coordinator to start
+            getContainingTask()
+                    .getEnvironment()
+                    .getOperatorCoordinatorEventGateway()
+                    .sendOperatorEventToCoordinator(
+                            getOperatorID(),
+                            new SerializedValue<>(
+                                    new StartEvent(counter.getLocalValue().intValue() - 1)));
+        }
+    }
+
+    private static class FailingCheckpointOperatorFactory<IN, OUT>

Review Comment:
   Should we have doc for this class, similar to how `EventReceivingOperatorFactory` is documented?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -392,8 +427,12 @@ public void setupSubtaskGatewayForAttempts(int subtask, Set<Integer> attemptNumb
     }
 
     private void setupSubtaskGateway(final SubtaskAccess sta) {
-        final OperatorCoordinator.SubtaskGateway gateway =
-                new SubtaskGatewayImpl(sta, eventValve, mainThreadExecutor, unconfirmedEvents);
+        final SubtaskGatewayImpl gateway =
+                new SubtaskGatewayImpl(sta, mainThreadExecutor, unconfirmedEvents);
+
+        if (!context.isConcurrentExecutionAttemptsSupported()) {

Review Comment:
   nits: would it be useful to add the following comment:
   
   When concurrent execution attempts is supported, the checkpoint must have been disabled and thus we don't need to maintain subtaskGatewayMap.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImplTest.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.operators.coordination.EventReceivingTasks.EventWithSubtask;
+import org.apache.flink.runtime.operators.coordination.util.IncompleteFuturesTracker;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for the {@link SubtaskGatewayImpl}. */
+public class SubtaskGatewayImplTest {
+
+    @Test
+    public void eventsPassThroughOpenGateway() {
+        final EventReceivingTasks receiver = EventReceivingTasks.createForRunningTasks();
+        final SubtaskGatewayImpl gateway =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(11)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+
+        final OperatorEvent event = new TestOperatorEvent();
+        final CompletableFuture<Acknowledge> future = gateway.sendEvent(event);
+
+        assertThat(receiver.events).containsExactly(new EventWithSubtask(event, 11));
+        assertThat(future).isDone();
+    }
+
+    @Test
+    public void closingMarkedGateway() {
+        final EventReceivingTasks receiver = EventReceivingTasks.createForRunningTasks();
+        final SubtaskGatewayImpl gateway =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(11)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+
+        gateway.markForCheckpoint(200L);
+        final boolean isClosed = gateway.tryCloseGateway(200L);
+
+        assertThat(isClosed).isTrue();
+    }
+
+    @Test
+    public void notClosingUnmarkedGateway() {
+        final EventReceivingTasks receiver = EventReceivingTasks.createForRunningTasks();
+        final SubtaskGatewayImpl gateway =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(11)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+
+        final boolean isClosed = gateway.tryCloseGateway(123L);
+
+        assertThat(isClosed).isFalse();
+    }
+
+    @Test
+    public void notClosingGatewayForOtherMark() {
+        final EventReceivingTasks receiver = EventReceivingTasks.createForRunningTasks();
+        final SubtaskGatewayImpl gateway =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(11)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+
+        gateway.markForCheckpoint(100L);
+        final boolean isClosed = gateway.tryCloseGateway(123L);
+
+        assertThat(isClosed).isFalse();
+    }
+
+    @Test
+    public void eventsBlockedByClosedGateway() {
+        final EventReceivingTasks receiver = EventReceivingTasks.createForRunningTasks();
+        final SubtaskGatewayImpl gateway =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(1)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+
+        gateway.markForCheckpoint(1L);
+        gateway.tryCloseGateway(1L);
+
+        final CompletableFuture<Acknowledge> future = gateway.sendEvent(new TestOperatorEvent());
+
+        assertThat(receiver.events).isEmpty();
+        assertThat(future).isNotDone();
+    }
+
+    @Test
+    public void eventsReleasedAfterOpeningGateway() {
+        final EventReceivingTasks receiver = EventReceivingTasks.createForRunningTasks();
+        final SubtaskGatewayImpl gateway0 =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(0)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+        final SubtaskGatewayImpl gateway3 =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(3)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+        List<SubtaskGatewayImpl> gateways = Arrays.asList(gateway3, gateway0);
+
+        gateways.forEach(x -> x.markForCheckpoint(17L));
+        gateways.forEach(x -> x.tryCloseGateway(17L));
+
+        final OperatorEvent event1 = new TestOperatorEvent();
+        final OperatorEvent event2 = new TestOperatorEvent();
+        final CompletableFuture<Acknowledge> future1 = gateway3.sendEvent(event1);
+        final CompletableFuture<Acknowledge> future2 = gateway0.sendEvent(event2);
+
+        gateways.forEach(SubtaskGatewayImpl::openGatewayAndUnmarkCheckpoint);
+
+        assertThat(receiver.events)
+                .containsExactly(new EventWithSubtask(event1, 3), new EventWithSubtask(event2, 0));
+        assertThat(future1).isDone();
+        assertThat(future2).isDone();
+    }
+
+    @Test
+    public void releasedEventsForwardSendFailures() {
+        final EventReceivingTasks receiver =
+                EventReceivingTasks.createForRunningTasksFailingRpcs(new FlinkException("test"));
+        final SubtaskGatewayImpl gateway =
+                new SubtaskGatewayImpl(
+                        getUniqueElement(receiver.getAccessesForSubtask(10)),
+                        ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+                        new IncompleteFuturesTracker());
+
+        gateway.markForCheckpoint(17L);
+        gateway.tryCloseGateway(17L);
+
+        final CompletableFuture<Acknowledge> future = gateway.sendEvent(new TestOperatorEvent());
+        gateway.openGatewayAndUnmarkCheckpoint();
+
+        assertThat(future).isCompletedExceptionally();
+    }
+
+    private static final class RejectingSubtaskGateway

Review Comment:
   Is this class used?



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint.
+ *
+ * <p>This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual

Review Comment:
   It is still not clear what `behavior` this class intends to test and how is the target behavior different from that tested by `CoordinatorEventsExactlyOnceITCase`.
   
   For example, it is not clear what is the problem with NOT using actual stream operators. And what `the correctness of the behavior of stream operators` actually means.
   
   The document of CoordinatorEventsExactlyOnceITCase has detailed comments explaining the behavior it is trying to verify. Can you follow that example and explain what behavior this test is trying to verify?
   
   Also note that the expected behavior described in CoordinatorEventsExactlyOnceITCase's doc is now outdated. It says `After a coordinator completed its checkpoint future, all events sent after that must be held back until the checkpoint barriers have been sent to the sources`, which is no longer sufficient for achieving exactly-once semantics. This probably also needs to be updated in this PR.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1202617613

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1208903178

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r931731203


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators.
+ */
+public class CoordinatorEventsExactlyOnceTest {

Review Comment:
   Because the implementations of this PR are in `flink-runtime` and `flink-streaming-java`. Given that `flink-streaming-java` depends on `flink-runtime`, an e2e test for the problem solved in this PR should be in `flink-streaming-java`.
   
   As already explained in the JavaDoc, The newly added test class verifies the correctness of coordinator events with actual stream operators, while the existing `CoordinatorEventsExactlyOnceITCase` makes the recipient of coordinator events a general `AbstractInvokable` subclass. The scope of these two test classes is different.
   
   How do you like it to remove `CoordinatorEventsExactlyOnceITCase`? I now think that the `AbstractInvokable` may not provide more verifications than a `StreamOperator`, so the test case can be removed. If we agree on this idea, there will not be two integration tests in two packages about a similar situation, and the naming problem would not exist.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r931934202


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators.
+ */
+public class CoordinatorEventsExactlyOnceTest {

Review Comment:
   According to our offline discussion, I'll give a more meaningful name to this test case and add corresponding descriptions in JavaDocs.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926212897


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {

Review Comment:
   I agree. I'll merge `CloseableSubtaskGateway` into `SubtaskGatewayImpl`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926209447


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {

Review Comment:
   I agree. I'll add the `checkRunsInMainThread()` to all methods that access or modify `isClosed` and send or block events.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r928509364


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -91,27 +130,133 @@ public CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt) {
                                                         new FlinkException(msg, failure)));
                             }
                         },
-                        sendingExecutor);
+                        mainThreadExecutor);
 
-        sendingExecutor.execute(
+        mainThreadExecutor.execute(
                 () -> {
-                    sender.sendEvent(sendAction, sendResult);
+                    sendEventInternal(sendAction, sendResult);
                     incompleteFuturesTracker.trackFutureWhileIncomplete(result);
                 });
+
         return result;
     }
 
-    @Override
-    public ExecutionAttemptID getExecution() {
-        return subtaskAccess.currentAttempt();
+    private void sendEventInternal(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        checkRunsInMainThread();
+
+        if (isClosed) {
+            blockedEvents.add(new BlockedEvent(sendAction, result));
+        } else {
+            callSendAction(sendAction, result);
+        }
     }
 
-    @Override
-    public int getSubtask() {
-        return subtaskAccess.getSubtaskIndex();
+    private void callSendAction(
+            Callable<CompletableFuture<Acknowledge>> sendAction,
+            CompletableFuture<Acknowledge> result) {
+        try {
+            final CompletableFuture<Acknowledge> sendResult = sendAction.call();
+            FutureUtils.forward(sendResult, result);
+        } catch (Throwable t) {
+            ExceptionUtils.rethrowIfFatalError(t);
+            result.completeExceptionally(t);
+        }
     }
 
-    private boolean isReady() {
-        return subtaskAccess.hasSwitchedToRunning().isDone();
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        checkRunsInMainThread();
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {

Review Comment:
   According to our offline discussion, the OC can receive the `AcknowledgeCheckpointEvent` of a previous checkpoint when it has closed gateways for a newer checkpoint, so in this case, the subtask gateway should ignore events about the previous checkpoint. I'll add the throw Exception code if `expectedCheckpointId > currentCheckpointId && currentCheckpointId != NO_CHECKPOINT`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r929775009


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -308,6 +337,24 @@ private void checkpointCoordinatorInternal(
         }
     }
 
+    private boolean closeGateways(
+            final long checkpointId, final Set<Integer> subtasksToCheckpoint) {
+        boolean hasCloseableGateway = false;
+        for (int subtask : subtasksToCheckpoint) {
+            SubtaskGatewayImpl gateway = subtaskGatewayMap.get(subtask);
+            if (!gateway.tryCloseGateway(checkpointId)) {

Review Comment:
   I agree that would be simpler. I'll make the change.
   
   If all gateways cannot be closed, it usually means that the checkpoint has been aborted before, so the future is already completed exceptionally. It is a possible situation that should not throw an exception. If some gateways can be closed while others cannot, which should never happen and once happens, it usually means a bug, we should throw exceptions in this situation.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -284,22 +311,24 @@ private void checkpointCoordinatorInternal(
                         (success, failure) -> {
                             if (failure != null) {
                                 result.completeExceptionally(failure);
-                            } else if (eventValve.tryShutValve(checkpointId)) {
+                            } else if (closeGateways(checkpointId, subtasksToCheckpoint)) {
                                 completeCheckpointOnceEventsAreDone(checkpointId, result, success);
                             } else {
-                                // if we cannot shut the valve, this means the checkpoint
+                                // if we cannot close the gateway, this means the checkpoint
                                 // has been aborted before, so the future is already
                                 // completed exceptionally. but we try to complete it here
                                 // again, just in case, as a safety net.
                                 result.completeExceptionally(
-                                        new FlinkException("Cannot shut event valve"));
+                                        new FlinkException("Cannot close gateway"));
                             }
                             return null;
                         },
                         mainThreadExecutor));
 
         try {
-            eventValve.markForCheckpoint(checkpointId);
+            for (int subtask : subtasksToCheckpoint) {
+                subtaskGatewayMap.get(subtask).markForCheckpoint(checkpointId);

Review Comment:
   Same as above.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r931719761


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -308,6 +337,24 @@ private void checkpointCoordinatorInternal(
         }
     }
 
+    private boolean closeGateways(
+            final long checkpointId, final Set<Integer> subtasksToCheckpoint) {
+        boolean hasCloseableGateway = false;
+        for (int subtask : subtasksToCheckpoint) {
+            SubtaskGatewayImpl gateway = subtaskGatewayMap.get(subtask);
+            if (!gateway.tryCloseGateway(checkpointId)) {

Review Comment:
   I agree. I'll make the change.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1208916714

   @flinkbot run azure
   
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1202489565

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926210207


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {
+            return;
+        }
+        openGatewayAndUnmarkCheckpoint();
+    }
+
+    /** Opens the gateway, releasing all buffered events. */
+    void openGatewayAndUnmarkCheckpoint() {

Review Comment:
   This method is only accessed by `OperatorCoordinatorHolder`, which is in the same package as this class, so the default package-private decorator is enough for this method. Similar practice also exists in Flink, like `OperatorEventDispatcherImpl.dispatchEventToHandlers()`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926212101


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -277,29 +300,33 @@ private void checkpointCoordinatorInternal(
             final long checkpointId, final CompletableFuture<byte[]> result) {
         mainThreadExecutor.assertRunningInMainThread();
 
-        final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
-
-        FutureUtils.assertNoException(
-                coordinatorCheckpoint.handleAsync(
-                        (success, failure) -> {
-                            if (failure != null) {
-                                result.completeExceptionally(failure);
-                            } else if (eventValve.tryShutValve(checkpointId)) {
-                                completeCheckpointOnceEventsAreDone(checkpointId, result, success);
-                            } else {
-                                // if we cannot shut the valve, this means the checkpoint
-                                // has been aborted before, so the future is already
-                                // completed exceptionally. but we try to complete it here
-                                // again, just in case, as a safety net.
-                                result.completeExceptionally(
-                                        new FlinkException("Cannot shut event valve"));
-                            }
-                            return null;
-                        },
-                        mainThreadExecutor));
-
         try {
-            eventValve.markForCheckpoint(checkpointId);
+            subtaskGatewayMap.values().forEach(x -> x.markForCheckpoint(checkpointId));
+
+            final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
+
+            coordinatorCheckpoint.whenComplete(
+                    (success, failure) -> {
+                        if (failure != null) {
+                            result.completeExceptionally(failure);
+                        } else {
+                            closeGateways(checkpointId, result);
+                        }
+                    });
+
+            FutureUtils.assertNoException(
+                    coordinatorCheckpoint.handleAsync(
+                            (success, failure) -> {
+                                if (failure != null) {
+                                    result.completeExceptionally(failure);

Review Comment:
   As we are removing the `whenComplete()` above and merging that logic to this `handleAsync()`, this comment is not applicable now.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r923952394


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorEventDispatcherImpl.java:
##########
@@ -82,6 +83,10 @@ public void registerEventHandler(OperatorID operator, OperatorEventHandler handl
         }
     }
 
+    Set<OperatorID> getRegisteredOperators() {

Review Comment:
   Could we explicitly mark methods as `private/protected/public` for readability and consistency?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -245,7 +262,13 @@ public void notifyCheckpointAborted(long checkpointId) {
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
-        mainThreadExecutor.execute(() -> coordinator.notifyCheckpointAborted(checkpointId));
+        mainThreadExecutor.execute(
+                () -> {
+                    subtaskGatewayMap
+                            .values()
+                            .forEach(x -> x.openGatewayAndUnmarkCheckpoint(checkpointId));
+                    coordinator.notifyCheckpointAborted(checkpointId);

Review Comment:
   This method does not call `eventValve.openValveAndUnmarkCheckpoint()` prior to this PR. Could you explain why it is now necessary to call `openGatewayAndUnmarkCheckpoint()`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -277,29 +300,33 @@ private void checkpointCoordinatorInternal(
             final long checkpointId, final CompletableFuture<byte[]> result) {
         mainThreadExecutor.assertRunningInMainThread();
 
-        final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
-
-        FutureUtils.assertNoException(
-                coordinatorCheckpoint.handleAsync(
-                        (success, failure) -> {
-                            if (failure != null) {
-                                result.completeExceptionally(failure);
-                            } else if (eventValve.tryShutValve(checkpointId)) {
-                                completeCheckpointOnceEventsAreDone(checkpointId, result, success);
-                            } else {
-                                // if we cannot shut the valve, this means the checkpoint
-                                // has been aborted before, so the future is already
-                                // completed exceptionally. but we try to complete it here
-                                // again, just in case, as a safety net.
-                                result.completeExceptionally(
-                                        new FlinkException("Cannot shut event valve"));
-                            }
-                            return null;
-                        },
-                        mainThreadExecutor));
-
         try {
-            eventValve.markForCheckpoint(checkpointId);
+            subtaskGatewayMap.values().forEach(x -> x.markForCheckpoint(checkpointId));
+
+            final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
+
+            coordinatorCheckpoint.whenComplete(
+                    (success, failure) -> {
+                        if (failure != null) {
+                            result.completeExceptionally(failure);
+                        } else {
+                            closeGateways(checkpointId, result);
+                        }
+                    });
+
+            FutureUtils.assertNoException(
+                    coordinatorCheckpoint.handleAsync(
+                            (success, failure) -> {
+                                if (failure != null) {
+                                    result.completeExceptionally(failure);

Review Comment:
   This line seems to duplicate the logic at line 311 above.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -308,6 +335,26 @@ private void checkpointCoordinatorInternal(
         }
     }
 
+    private void closeGateways(final long checkpointId, final CompletableFuture<byte[]> result) {
+        Set<CloseableSubtaskGateway> closedGateways = new HashSet<>();
+        for (CloseableSubtaskGateway gateway : subtaskGatewayMap.values()) {
+            if (!gateway.tryCloseGateway(checkpointId)) {
+                closedGateways.forEach(CloseableSubtaskGateway::openGatewayAndUnmarkCheckpoint);

Review Comment:
   Is there any case where `closedGateways` is not empty at this point? If no, could it be better to throw IllegalStateException if `closedGateways()` is not empty?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r928498354


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java:
##########
@@ -18,43 +18,82 @@
 
 package org.apache.flink.runtime.operators.coordination;
 
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.operators.coordination.util.IncompleteFuturesTracker;
 import org.apache.flink.runtime.util.Runnables;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.concurrent.FutureUtils;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
 
 /**
  * Implementation of the {@link OperatorCoordinator.SubtaskGateway} interface that access to
  * subtasks for status and event sending via {@link SubtaskAccess}.
+ *
+ * <p>Instances of this class can be temporarily closed, blocking events from going through,
+ * buffering them, and releasing them later. It is used for "alignment" of operator event streams
+ * with checkpoint barrier injection, similar to how the input channels are aligned during a common
+ * checkpoint.
+ *
+ * <p>The methods on the critical communication path, including closing/reopening the gateway and
+ * sending the operator events, are required to be used in a single-threaded context specified by
+ * the {@link #mainThreadExecutor} in {@link #SubtaskGatewayImpl(SubtaskAccess,
+ * ComponentMainThreadExecutor, IncompleteFuturesTracker)} in order to avoid concurrent issues.
  */
 class SubtaskGatewayImpl implements OperatorCoordinator.SubtaskGateway {
 
     private static final String EVENT_LOSS_ERROR_MESSAGE =
             "An OperatorEvent from an OperatorCoordinator to a task was lost. "
                     + "Triggering task failover to ensure consistency. Event: '%s', targetTask: %s";
 
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
     private final SubtaskAccess subtaskAccess;
-    private final EventSender sender;
-    private final Executor sendingExecutor;
+
+    private final ComponentMainThreadExecutor mainThreadExecutor;
+
     private final IncompleteFuturesTracker incompleteFuturesTracker;
 
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
     SubtaskGatewayImpl(
             SubtaskAccess subtaskAccess,
-            EventSender sender,
-            Executor sendingExecutor,
+            ComponentMainThreadExecutor mainThreadExecutor,
             IncompleteFuturesTracker incompleteFuturesTracker) {
         this.subtaskAccess = subtaskAccess;
-        this.sender = sender;
-        this.sendingExecutor = sendingExecutor;
+        this.mainThreadExecutor = mainThreadExecutor;
         this.incompleteFuturesTracker = incompleteFuturesTracker;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    @Override
+    public ExecutionAttemptID getExecution() {

Review Comment:
   There is no important reason to move these methods. I'll move them back.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r937314000


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 10;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        ManuallyClosedSourceFunction.isCloseSource = false;
+        EventSendingCoordinatorWithGuaranteedCheckpoint.isEventSentAfterFirstCheckpoint = false;
+    }
+
+    @Test
+    public void testCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testCheckpointWithSubtaskFailure() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testUnalignedCheckpointWithSubtaskFailure() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new ManuallyClosedSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and closes on demand. */
+    private static class ManuallyClosedSourceFunction<T> implements SourceFunction<T> {
+
+        /** Whether the source function should be closed to finish the job. */
+        private static boolean isCloseSource;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the first checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();

Review Comment:
   I thought about it and agree that it is a good idea to use `output.collect(element)` here to simplify the additional behavior introduced by `BlockCheckpointBarrierOperator`.
   
   As for the other `throw new UnsupportedOperationException();` in `EventReceivingOperator.processElement()`, I'll add more comments here to notify readers that this method should never be invoked, instead of leaving ambiguity that it might be expected for this method to be invoked and throw exceptions.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r931714614


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsExactlyOnceTest.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.MiniClusterResource;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EndEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.EventSendingCoordinator;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.IntegerEvent;
+import static org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase.StartEvent;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link
+ * org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events when the flink job is constructed using actual
+ * stream operators.
+ */
+public class CoordinatorEventsExactlyOnceTest {
+    @ClassRule
+    public static final MiniClusterResource MINI_CLUSTER =
+            new MiniClusterResource(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setNumberSlotsPerTaskManager(2)
+                            .build());
+
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void test() throws Exception {
+        long count =
+                executeAndGetNumReceivedEvents(
+                        env,
+                        new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        long count =
+                executeAndGetNumReceivedEvents(
+                        env,
+                        new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {

Review Comment:
   In a previous comment, we discussed whether there is a duplication to open gateways both in `abortCurrentTriggering()` and `notifyCheckpointAborted()` methods. This test verifies that it is necessary to open gateways in `notifyCheckpointAborted()`, or the Flink job in this test case would fail or run infinitely.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r935029768


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new FailingCheckpointOperatorFactory<>("failingCheckpoint", numEvents, delay));
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, OneInputStreamOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // Differs the parallelism of source operators from that of tested operators, so that when
+        // checkpoint barriers are injected into sources, the tested operators have not started
+        // checkpoint yet.
+        DataStream<Long> stream =
+                env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                        .setParallelism(2);
+        stream =
+                stream.transform("eventReceiving", TypeInformation.of(Long.class), factory)
+                        .setParallelism(1);
+        stream.addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        long count = executionResult.getAccumulatorResult(EventReceivingOperator.COUNTER_NAME);
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        private boolean isCancelled = false;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCancelled && !shouldCloseSource.get()) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {
+            isCancelled = true;
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+        private final String name;
+        protected final int numEvents;
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * The stream operator that receives the events and accumulates the numbers. The task is
+     * stateful and checkpoints the accumulator.
+     */
+    private static class EventReceivingOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T>, OperatorEventHandler {
+        protected static final String COUNTER_NAME = "numEvents";
+
+        protected final LongCounter counter = new LongCounter();
+
+        protected ListState<Long> state;
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+            getRuntimeContext().addAccumulator(COUNTER_NAME, counter);
+        }
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                shouldCloseSource.set(true);
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            state.update(Collections.singletonList(counter.getLocalValue()));
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+
+            state =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "counterState", BasicTypeInfo.LONG_TYPE_INFO));
+
+            counter.resetLocal();
+            Iterator<Long> iterator = state.get().iterator();
+            if (iterator.hasNext()) {
+                counter.add(iterator.next());
+            }
+            Preconditions.checkArgument(!iterator.hasNext());
+
+            // signal the coordinator to start
+            getContainingTask()
+                    .getEnvironment()
+                    .getOperatorCoordinatorEventGateway()
+                    .sendOperatorEventToCoordinator(
+                            getOperatorID(),
+                            new SerializedValue<>(
+                                    new StartEvent(counter.getLocalValue().intValue() - 1)));
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link FailingCheckpointOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class FailingCheckpointOperatorFactory<IN, OUT>
+            extends EventReceivingOperatorFactory<IN, OUT> {
+        public FailingCheckpointOperatorFactory(String name, int numEvents, int delay) {
+            super(name, numEvents, delay);
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new FailingCheckpointOperator<>(numEvents);
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+    }
+
+    /**
+     * A subclass of {@link EventReceivingOperator} whose subtask would fail during a specific
+     * checkpoint.
+     */
+    private static class FailingCheckpointOperator<T> extends EventReceivingOperator<T> {
+
+        private final int failAtCheckpointAfterMessage;
+
+        private TestScript testScript;
+
+        private FailingCheckpointOperator(int numEvents) {
+            this.failAtCheckpointAfterMessage =
+                    numEvents * 2 / 3 + new Random().nextInt(numEvents / 6);
+        }
+
+        @Override
+        public void setup(
+                StreamTask<?, ?> containingTask,
+                StreamConfig config,
+                Output<StreamRecord<T>> output) {
+            super.setup(containingTask, config, output);
+            if (containingTask.getIndexInSubtaskGroup() == 0) {
+                this.testScript = TestScript.getForOperator(getOperatorID() + "-subtask0");
+            } else {
+                this.testScript = null;
+            }
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                if (testScript.hasAlreadyFailed()) {
+                    shouldCloseSource.set(true);
+                }
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            if (counter.getLocalValue() > failAtCheckpointAfterMessage
+                    && testScript != null
+                    && !testScript.hasAlreadyFailed()) {
+                testScript.recordHasFailed();
+                throw new RuntimeException();

Review Comment:
   It is guaranteed that the exception will be thrown before the test job ends, or `shouldCloseSource` would not be set to true. I'll also add an `assertThat` command to verify this point.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r935026305


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+    private static final AtomicBoolean shouldCloseSource = new AtomicBoolean(false);
+
+    private static final int numEvents = 100;
+
+    private static final int delay = 1;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        shouldCloseSource.set(false);
+    }
+
+    @Test
+    public void testCoordinatorSendEvents() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", numEvents, delay));
+    }
+
+    @Test
+    public void testFailingCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new FailingCheckpointOperatorFactory<>("failingCheckpoint", numEvents, delay));
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, OneInputStreamOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // Differs the parallelism of source operators from that of tested operators, so that when
+        // checkpoint barriers are injected into sources, the tested operators have not started
+        // checkpoint yet.
+        DataStream<Long> stream =
+                env.addSource(new IdlingSourceFunction<>(), TypeInformation.of(Long.class))
+                        .setParallelism(2);
+        stream =
+                stream.transform("eventReceiving", TypeInformation.of(Long.class), factory)
+                        .setParallelism(1);
+        stream.addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        long count = executionResult.getAccumulatorResult(EventReceivingOperator.COUNTER_NAME);
+        assertThat(count).isEqualTo(numEvents);
+    }
+
+    /** A mock source function that does not collect any stream record and finishes on demand. */
+    private static class IdlingSourceFunction<T> extends RichSourceFunction<T>
+            implements ParallelSourceFunction<T> {
+        private boolean isCancelled = false;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCancelled && !shouldCloseSource.get()) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {
+            isCancelled = true;
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link EventReceivingOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class EventReceivingOperatorFactory<IN, OUT>
+            extends AbstractStreamOperatorFactory<OUT>
+            implements CoordinatedOperatorFactory<OUT>, OneInputStreamOperatorFactory<IN, OUT> {
+        private final String name;
+        protected final int numEvents;
+        private final int delay;
+
+        public EventReceivingOperatorFactory(String name, int numEvents, int delay) {
+            this.name = name;
+            this.numEvents = numEvents;
+            this.delay = delay;
+        }
+
+        @Override
+        public OperatorCoordinator.Provider getCoordinatorProvider(
+                String operatorName, OperatorID operatorID) {
+            return new OperatorCoordinator.Provider() {
+
+                @Override
+                public OperatorID getOperatorId() {
+                    return operatorID;
+                }
+
+                @Override
+                public OperatorCoordinator create(OperatorCoordinator.Context context) {
+                    return new EventSendingCoordinator(context, name, numEvents, delay);
+                }
+            };
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new EventReceivingOperator<>();
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+
+        @Override
+        public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) {
+            return EventReceivingOperator.class;
+        }
+    }
+
+    /**
+     * The stream operator that receives the events and accumulates the numbers. The task is
+     * stateful and checkpoints the accumulator.
+     */
+    private static class EventReceivingOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T>, OperatorEventHandler {
+        protected static final String COUNTER_NAME = "numEvents";
+
+        protected final LongCounter counter = new LongCounter();
+
+        protected ListState<Long> state;
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+            getRuntimeContext().addAccumulator(COUNTER_NAME, counter);
+        }
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                shouldCloseSource.set(true);
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            state.update(Collections.singletonList(counter.getLocalValue()));
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+
+            state =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "counterState", BasicTypeInfo.LONG_TYPE_INFO));
+
+            counter.resetLocal();
+            Iterator<Long> iterator = state.get().iterator();
+            if (iterator.hasNext()) {
+                counter.add(iterator.next());
+            }
+            Preconditions.checkArgument(!iterator.hasNext());
+
+            // signal the coordinator to start
+            getContainingTask()
+                    .getEnvironment()
+                    .getOperatorCoordinatorEventGateway()
+                    .sendOperatorEventToCoordinator(
+                            getOperatorID(),
+                            new SerializedValue<>(
+                                    new StartEvent(counter.getLocalValue().intValue() - 1)));
+        }
+    }
+
+    /**
+     * A wrapper operator factory for {@link FailingCheckpointOperator} and {@link
+     * EventSendingCoordinator}.
+     */
+    private static class FailingCheckpointOperatorFactory<IN, OUT>
+            extends EventReceivingOperatorFactory<IN, OUT> {
+        public FailingCheckpointOperatorFactory(String name, int numEvents, int delay) {
+            super(name, numEvents, delay);
+        }
+
+        @Override
+        public <T extends StreamOperator<OUT>> T createStreamOperator(
+                StreamOperatorParameters<OUT> parameters) {
+            EventReceivingOperator<OUT> operator = new FailingCheckpointOperator<>(numEvents);
+            operator.setup(
+                    parameters.getContainingTask(),
+                    parameters.getStreamConfig(),
+                    parameters.getOutput());
+            parameters
+                    .getOperatorEventDispatcher()
+                    .registerEventHandler(parameters.getStreamConfig().getOperatorID(), operator);
+            return (T) operator;
+        }
+    }
+
+    /**
+     * A subclass of {@link EventReceivingOperator} whose subtask would fail during a specific
+     * checkpoint.
+     */
+    private static class FailingCheckpointOperator<T> extends EventReceivingOperator<T> {
+
+        private final int failAtCheckpointAfterMessage;
+
+        private TestScript testScript;
+
+        private FailingCheckpointOperator(int numEvents) {
+            this.failAtCheckpointAfterMessage =
+                    numEvents * 2 / 3 + new Random().nextInt(numEvents / 6);
+        }
+
+        @Override
+        public void setup(
+                StreamTask<?, ?> containingTask,
+                StreamConfig config,
+                Output<StreamRecord<T>> output) {
+            super.setup(containingTask, config, output);
+            if (containingTask.getIndexInSubtaskGroup() == 0) {
+                this.testScript = TestScript.getForOperator(getOperatorID() + "-subtask0");
+            } else {
+                this.testScript = null;
+            }
+        }
+
+        @Override
+        public void handleOperatorEvent(OperatorEvent evt) {
+            if (evt instanceof IntegerEvent) {
+                counter.add(1L);
+            } else if (evt instanceof EndEvent) {
+                try {
+                    state.update(Collections.singletonList(counter.getLocalValue()));
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                if (testScript.hasAlreadyFailed()) {

Review Comment:
   It seems that the parallelism of this operator has to be 1 in order to work correctly, thus `testScript` must have been assigned a value during setup. I'll modify the logic here.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 closed pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 closed pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks
URL: https://github.com/apache/flink/pull/20275


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1184371295

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e8605861d236de75c5c7fee88d181616f6072e30",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e8605861d236de75c5c7fee88d181616f6072e30",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e8605861d236de75c5c7fee88d181616f6072e30 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1204892369

   @yunfengzhou-hub Thanks for the update. LGTM.
   
   @becketqin Would you like to take a look at this PR?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r937419550


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase.java:
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.operators.coordination.CoordinatorEventsExactlyOnceITCase;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration test case that validates the exactly-once mechanism for operator events sent around
+ * checkpoint. This class is an extension to {@link CoordinatorEventsExactlyOnceITCase}, further
+ * verifying the exactly-once semantics of events in the following conditions:
+ *
+ * <h2>Stream operator recipient</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the test cases focus on verifying the
+ * correctness of operator coordinator's behavior. It uses a custom {@link AbstractInvokable}
+ * subclass that mocks the behavior of coordinator's recipients. This test class uses actual stream
+ * operators as the recipient of coordinator events, verifying that stream operators can correctly
+ * handle received operator events and inform coordinators of completing checkpoint.
+ *
+ * <h2>Non-source stream task</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, the two tested tasks are executed independently
+ * of each other. They do not have the upstream-downstream relationship as operators usually do in a
+ * streaming job, and thus both of them are treated as source tasks. This test class further
+ * verifies situations when the tested operators are not sources, which means when checkpoint
+ * barriers are injected into sources, these operators may not have started checkpoint yet.
+ *
+ * <h2>Unaligned checkpoint</h2>
+ *
+ * <p>This class tests both aligned and unaligned checkpoints to verify that the correctness of the
+ * event delivery behavior around checkpoint is not affected by this condition.
+ *
+ * <h2>Non-global failure</h2>
+ *
+ * <p>In {@link CoordinatorEventsExactlyOnceITCase}, failures occur at the coordinators' side, so
+ * they will cause the whole Flink job to fail over. In this class, test cases are added when there
+ * might only be fail-overs on the subtasks' side, while the coordinators are not affected. In this
+ * case the production infrastructure code needs to work together with user code (implementations
+ * inside the coordinator and operator subclass) to ensure the exactly-once semantics of operator
+ * event delivery.
+ *
+ * <p>In the test cases of this class, the tested coordinator would inject failure during its
+ * sending operator events. Besides, it is additionally guaranteed that there must have been a
+ * checkpoint completed before the failure is injected, and that there must be events sent from the
+ * coordinator to its subtask during checkpoint.
+ */
+public class CoordinatorEventsToStreamOperatorRecipientExactlyOnceITCase
+        extends CoordinatorEventsExactlyOnceITCase {
+
+    private static final int NUM_EVENTS = 100;
+
+    private static final int DELAY = 10;
+
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void setup() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        ManuallyClosedSourceFunction.isCloseSource = false;
+        EventSendingCoordinatorWithGuaranteedCheckpoint.isEventSentAfterFirstCheckpoint = false;
+    }
+
+    @Test
+    public void testCheckpoint() throws Exception {
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testUnalignedCheckpoint() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env, new EventReceivingOperatorFactory<>("eventReceiving", NUM_EVENTS, DELAY));
+    }
+
+    @Test
+    public void testCheckpointWithSubtaskFailure() throws Exception {
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    @Test
+    public void testUnalignedCheckpointWithSubtaskFailure() throws Exception {
+        env.getCheckpointConfig().enableUnalignedCheckpoints();
+        executeAndVerifyResults(
+                env,
+                new EventReceivingOperatorWithFailureFactory<>(
+                        "eventReceivingWithFailure", NUM_EVENTS, DELAY));
+        assertThat(
+                        TestScript.getForOperator("eventReceivingWithFailure-subtask0")
+                                .hasAlreadyFailed())
+                .isTrue();
+    }
+
+    private void executeAndVerifyResults(
+            StreamExecutionEnvironment env, EventReceivingOperatorFactory<Long, Long> factory)
+            throws Exception {
+        // The event receiving operator is not chained together with the source operator, so that
+        // when checkpoint barriers are injected into sources, the event receiving operator has not
+        // started checkpoint yet.
+        env.addSource(new ManuallyClosedSourceFunction<>(), TypeInformation.of(Long.class))
+                .transform(
+                        "blockCheckpointBarrier",
+                        TypeInformation.of(Long.class),
+                        new BlockCheckpointBarrierOperator<>())
+                .disableChaining()
+                .transform(factory.name, TypeInformation.of(Long.class), factory)
+                .addSink(new DiscardingSink<>());
+
+        JobExecutionResult executionResult =
+                MINI_CLUSTER
+                        .getMiniCluster()
+                        .executeJobBlocking(env.getStreamGraph().getJobGraph());
+
+        List<Integer> receivedInts =
+                executionResult.getAccumulatorResult(EventReceivingOperator.ACCUMULATOR_NAME);
+        checkListContainsSequence(receivedInts, NUM_EVENTS);
+    }
+
+    /** A mock source function that does not collect any stream record and closes on demand. */
+    private static class ManuallyClosedSourceFunction<T> implements SourceFunction<T> {
+
+        /** Whether the source function should be closed to finish the job. */
+        private static boolean isCloseSource;
+
+        @Override
+        public void run(SourceContext<T> ctx) throws Exception {
+            while (!isCloseSource) {
+                Thread.sleep(100);
+            }
+        }
+
+        @Override
+        public void cancel() {}
+    }
+
+    /**
+     * A stream operator that blocks the checkpoint barrier until the coordinator has sent events to
+     * its subtask. It helps to guarantee that there are events being sent when the coordinator has
+     * completed the first checkpoint while the subtask has not yet.
+     */
+    private static class BlockCheckpointBarrierOperator<T> extends AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T> {
+
+        @Override
+        public void processElement(StreamRecord<T> element) throws Exception {
+            throw new UnsupportedOperationException();

Review Comment:
   Sounds good.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
lindong28 commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1209321806

   Thanks for the update! This PR has been merged to the master branch.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r926266725


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -308,6 +335,26 @@ private void checkpointCoordinatorInternal(
         }
     }
 
+    private void closeGateways(final long checkpointId, final CompletableFuture<byte[]> result) {
+        Set<CloseableSubtaskGateway> closedGateways = new HashSet<>();
+        for (CloseableSubtaskGateway gateway : subtaskGatewayMap.values()) {
+            if (!gateway.tryCloseGateway(checkpointId)) {
+                closedGateways.forEach(CloseableSubtaskGateway::openGatewayAndUnmarkCheckpoint);

Review Comment:
   I agree. I'll make it throw exception here.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on PR #20275:
URL: https://github.com/apache/flink/pull/20275#issuecomment-1195311943

   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] lindong28 commented on a diff in pull request #20275: [FLINK-26029][Runtime/Checkpointing] Generalize the checkpoint protocol of OperatorCoordinator

Posted by GitBox <gi...@apache.org>.
lindong28 commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r923461372


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {

Review Comment:
   According to the Java doc of this class, `This class is NOT thread safe, but assumed to be used in a single threaded context`. Should we also invoke `checkRunsInMainThread()` in methods such as `tryCloseGateway()` and `sendEvent()`?
   
   I assume we want to make sure tha `isClosed` and `blockedEvents` won't be accessed concurrently.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -236,7 +240,20 @@ public void notifyCheckpointComplete(long checkpointId) {
         // checkpoint coordinator time thread.
         // we can remove the delegation once the checkpoint coordinator runs fully in the
         // scheduler's main thread executor
-        mainThreadExecutor.execute(() -> coordinator.notifyCheckpointComplete(checkpointId));
+        mainThreadExecutor.execute(
+                () -> {
+                    coordinator.notifyCheckpointComplete(checkpointId);
+                    for (Map.Entry<Integer, CloseableSubtaskGateway> entry :
+                            subtaskGatewayMap.entrySet()) {
+                        if (entry.getValue().isClosed()) {
+                            LOG.warn(

Review Comment:
   Is there any case that this would happen? If no, would it be simpler to throw exception?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {

Review Comment:
   Typically `CloseableXXX` refers to classes that implement the `java.io.Closeable` interface. Could we rename this class to reduce confusion?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -277,29 +300,33 @@ private void checkpointCoordinatorInternal(
             final long checkpointId, final CompletableFuture<byte[]> result) {
         mainThreadExecutor.assertRunningInMainThread();
 
-        final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
-
-        FutureUtils.assertNoException(
-                coordinatorCheckpoint.handleAsync(
-                        (success, failure) -> {
-                            if (failure != null) {
-                                result.completeExceptionally(failure);
-                            } else if (eventValve.tryShutValve(checkpointId)) {
-                                completeCheckpointOnceEventsAreDone(checkpointId, result, success);
-                            } else {
-                                // if we cannot shut the valve, this means the checkpoint
-                                // has been aborted before, so the future is already
-                                // completed exceptionally. but we try to complete it here
-                                // again, just in case, as a safety net.
-                                result.completeExceptionally(
-                                        new FlinkException("Cannot shut event valve"));
-                            }
-                            return null;
-                        },
-                        mainThreadExecutor));
-
         try {
-            eventValve.markForCheckpoint(checkpointId);
+            subtaskGatewayMap.values().forEach(x -> x.markForCheckpoint(checkpointId));
+
+            final CompletableFuture<byte[]> coordinatorCheckpoint = new CompletableFuture<>();
+
+            coordinatorCheckpoint.whenComplete(
+                    (success, failure) -> {
+                        if (failure != null) {
+                            result.completeExceptionally(failure);
+                        } else {
+                            closeGateways(checkpointId, result);

Review Comment:
   `coordinatorCheckpoint.whenComplete(...)` does not guarantee the provided `action` is invoked by the current thread. In order to make sure `closeGateways()` can still be executed using `mainThreadExecutor`, it seems necessary to use `handleAsync(...)` as did before this PR.
   



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {
+            return;
+        }
+        openGatewayAndUnmarkCheckpoint();
+    }
+
+    /** Opens the gateway, releasing all buffered events. */
+    void openGatewayAndUnmarkCheckpoint() {

Review Comment:
   Would it be more readable to mark this method as either public or private? Same for other methods.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {
+        checkRunsInMainThread();
+
+        if (expectedCheckpointId != currentCheckpointId) {
+            return;
+        }
+        openGatewayAndUnmarkCheckpoint();
+    }
+
+    /** Opens the gateway, releasing all buffered events. */
+    void openGatewayAndUnmarkCheckpoint() {
+        checkRunsInMainThread();
+
+        currentCheckpointId = NO_CHECKPOINT;
+        if (!isClosed) {
+            return;
+        }
+
+        for (BlockedEvent blockedEvent : blockedEvents) {
+            CompletableFuture<Acknowledge> result =
+                    innerGateway.sendEvent(blockedEvent.operatorEvent);
+            FutureUtils.forward(result, blockedEvent.future);
+        }
+        blockedEvents.clear();
+
+        isClosed = false;
+    }
+
+    @Override
+    public CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt) {
+        if (isClosed) {
+            CompletableFuture<Acknowledge> sendResult = new CompletableFuture<>();

Review Comment:
   Suppose the given operator event can not be serialized and the gateway is closed, the exception will be added to the future returned to the caller. 
   
   This behavior is inconsistent with the case when the gateway is open. Also note that many callers currently do not check the future returned by this method. This means that an exception that can be caught previously might be missed now.
   
   Would it be safer to keep the previous behavior by always throwing the deserialization exception directly to the caller?
   
   Same for the exception that is thrown when the gateway is not read.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CloseableSubtaskGateway.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An {@link org.apache.flink.runtime.operators.coordination.OperatorCoordinator.SubtaskGateway}
+ * that can be temporarily closed, blocking events from going through, buffering them, and releasing
+ * them later. It is used for "alignment" of operator event streams with checkpoint barrier
+ * injection, similar to how the input channels are aligned during a common checkpoint.
+ *
+ * <p>This class is NOT thread safe, but assumed to be used in a single threaded context. To guard
+ * that, one can register a "main thread executor" (as used by the mailbox components like RPC
+ * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}.
+ */
+class CloseableSubtaskGateway implements OperatorCoordinator.SubtaskGateway {
+
+    /** The wrapped gateway that actually performs the event-sending operation. */
+    private final OperatorCoordinator.SubtaskGateway innerGateway;
+
+    private static final long NO_CHECKPOINT = Long.MIN_VALUE;
+
+    private final List<BlockedEvent> blockedEvents = new ArrayList<>();
+
+    private long currentCheckpointId;
+
+    private long lastCheckpointId;
+
+    private boolean isClosed;
+
+    @Nullable private ComponentMainThreadExecutor mainThreadExecutor;
+
+    CloseableSubtaskGateway(OperatorCoordinator.SubtaskGateway innerGateway) {
+        this.innerGateway = innerGateway;
+        this.currentCheckpointId = NO_CHECKPOINT;
+        this.lastCheckpointId = Long.MIN_VALUE;
+    }
+
+    void setMainThreadExecutorForValidation(ComponentMainThreadExecutor mainThreadExecutor) {
+        this.mainThreadExecutor = mainThreadExecutor;
+    }
+
+    /**
+     * Marks the gateway for the next checkpoint. This remembers the checkpoint ID and will only
+     * allow closing the gateway for this specific checkpoint.
+     *
+     * <p>This is the gateway's mechanism to detect situations where multiple coordinator
+     * checkpoints would be attempted overlapping, which is currently not supported (the gateway
+     * doesn't keep a list of events blocked per checkpoint). It also helps to identify situations
+     * where the checkpoint was aborted even before the gateway was closed (by finding out that the
+     * {@code currentCheckpointId} was already reset to {@code NO_CHECKPOINT}.
+     */
+    void markForCheckpoint(long checkpointId) {
+        checkRunsInMainThread();
+
+        if (currentCheckpointId != NO_CHECKPOINT && currentCheckpointId != checkpointId) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Cannot mark for checkpoint %d, already marked for checkpoint %d",
+                            checkpointId, currentCheckpointId));
+        }
+        if (checkpointId > lastCheckpointId) {
+            currentCheckpointId = checkpointId;
+            lastCheckpointId = checkpointId;
+        } else {
+            throw new IllegalStateException(
+                    String.format(
+                            "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d",
+                            lastCheckpointId, checkpointId));
+        }
+    }
+
+    /**
+     * Closes the gateway. All events sent through this gateway are blocked until the gateway is
+     * re-opened. If the gateway is already closed, this does nothing.
+     *
+     * @return True if the gateway is closed, false if the checkpointId is incorrect.
+     */
+    boolean tryCloseGateway(long checkpointId) {
+        if (checkpointId == currentCheckpointId) {
+            isClosed = true;
+            return true;
+        }
+        return false;
+    }
+
+    boolean isClosed() {
+        return isClosed;
+    }
+
+    void openGatewayAndUnmarkCheckpoint(long expectedCheckpointId) {

Review Comment:
   Do we expect the communication from coordinator to operator to always support `openGatewayAndUnmarkCheckpoint`? 
   
   If yes, would it be more reasonable o put such methods in the `SubtaskGateway` and merge `CloseableSubtaskGateway` into `SubtaskGatewayImpl`?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] yunfengzhou-hub commented on a diff in pull request #20275: [FLINK-28606][Runtime/Checkpointing] Preserve consistency of OperatorEvent from OperatorCoordinator to subtasks

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on code in PR #20275:
URL: https://github.com/apache/flink/pull/20275#discussion_r928496513


##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventDispatcher.java:
##########
@@ -29,6 +29,9 @@ public interface OperatorEventDispatcher {
     /**
      * Register a listener that is notified every time an OperatorEvent is sent from the
      * OperatorCoordinator (of the operator with the given OperatorID) to this subtask.
+     *
+     * <p>The stream operator with the given OperatorID must implement {@link OperatorEventHandler}

Review Comment:
   Because our consistency mechanism requires communications in both directions, all operators with the corresponding OC must be registered as handlers.
   
   This is not required for implementations in this PR and may be used in our solution's second phase to ticket FLINK-26029. So I'll remove this doc for now and possibly add it back in the next PR.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org