StephanEwen commented on a change in pull request #15601:
URL: https://github.com/apache/flink/pull/15601#discussion_r614199702



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import 
org.apache.flink.runtime.operators.coordination.util.IncompleteFuturesTracker;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.SerializedValue;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+final class ExecutionSubtaskAccess implements SubtaskAccess {
+
+    private final Execution taskExecution;
+    private final OperatorID operator;
+    private final IncompleteFuturesTracker futuresTracker;
+
+    ExecutionSubtaskAccess(Execution taskExecution, OperatorID operator) {
+        this.taskExecution = taskExecution;
+        this.operator = operator;
+        this.futuresTracker = new IncompleteFuturesTracker();
+
+        // this is a safeguard to speed up things: as soon as the task is in a 
terminal state, all
+        // the pending futures from events sent to that task should fail 
immediately
+        taskExecution
+                .getTerminalStateFuture()
+                .thenAccept(
+                        (state) ->
+                                futuresTracker.failAllFutures(
+                                        new FlinkException("Task is no longer 
running")));
+    }
+
+    @Override
+    public Callable<CompletableFuture<Acknowledge>> createEventSendAction(
+            SerializedValue<OperatorEvent> event) {
+        return () -> {
+            final ExecutionState state = taskExecution.getState();
+            if (!(state == ExecutionState.RUNNING || state == 
ExecutionState.INITIALIZING)) {

Review comment:
       Let me check if we can remove this, rely on only the other place.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java
##########
@@ -183,6 +184,8 @@ default void notifyCheckpointAborted(long checkpointId) {}
      */
     void subtaskReset(int subtask, long checkpointId);
 
+    void subtaskReady(int subtask, SubtaskGateway gateway);

Review comment:
       Will extend the 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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to