tillrohrmann commented on a change in pull request #15342:
URL: https://github.com/apache/flink/pull/15342#discussion_r599590545
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/OperatorCoordinatorHandler.java
##########
@@ -19,129 +19,28 @@
package org.apache.flink.runtime.scheduler;
import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
-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.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
-import
org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
-import
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
-import org.apache.flink.runtime.operators.coordination.TaskNotRunningException;
-import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.IOUtils;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
/** Handler for the {@link OperatorCoordinator OperatorCoordinators}. */
-public class OperatorCoordinatorHandler {
- private final ExecutionGraph executionGraph;
+public interface OperatorCoordinatorHandler {
+ void initializeOperatorCoordinators(ComponentMainThreadExecutor
mainThreadExecutor);
- private final Map<OperatorID, OperatorCoordinatorHolder> coordinatorMap;
+ void startAllOperatorCoordinators();
- private final Consumer<Throwable> globalFailureHandler;
+ void disposeAllOperatorCoordinators();
- public OperatorCoordinatorHandler(
- ExecutionGraph executionGraph, Consumer<Throwable>
globalFailureHandler) {
- this.executionGraph = executionGraph;
+ void deliverOperatorEventToCoordinator(
+ ExecutionAttemptID taskExecutionId, OperatorID operatorId,
OperatorEvent evt)
+ throws FlinkException;
- this.coordinatorMap = createCoordinatorMap(executionGraph);
- this.globalFailureHandler = globalFailureHandler;
- }
-
- private Map<OperatorID, OperatorCoordinatorHolder> createCoordinatorMap(
- ExecutionGraph executionGraph) {
- Map<OperatorID, OperatorCoordinatorHolder> coordinatorMap = new
HashMap<>();
- for (ExecutionJobVertex vertex :
executionGraph.getAllVertices().values()) {
- for (OperatorCoordinatorHolder holder :
vertex.getOperatorCoordinators()) {
- coordinatorMap.put(holder.operatorId(), holder);
- }
- }
- return coordinatorMap;
- }
-
- public void initializeOperatorCoordinators(ComponentMainThreadExecutor
mainThreadExecutor) {
- for (OperatorCoordinatorHolder coordinatorHolder :
coordinatorMap.values()) {
- coordinatorHolder.lazyInitialize(globalFailureHandler,
mainThreadExecutor);
- }
- }
-
- public void startAllOperatorCoordinators() {
- final Collection<OperatorCoordinatorHolder> coordinators =
coordinatorMap.values();
- try {
- for (OperatorCoordinatorHolder coordinator : coordinators) {
- coordinator.start();
- }
- } catch (Throwable t) {
- ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
- coordinators.forEach(IOUtils::closeQuietly);
- throw new FlinkRuntimeException("Failed to start the operator
coordinators", t);
- }
- }
-
- public void disposeAllOperatorCoordinators() {
- coordinatorMap.values().forEach(IOUtils::closeQuietly);
- }
-
- public void deliverOperatorEventToCoordinator(
- final ExecutionAttemptID taskExecutionId,
- final OperatorID operatorId,
- final OperatorEvent evt)
- throws FlinkException {
-
- // Failure semantics (as per the javadocs of the method):
- // If the task manager sends an event for a non-running task or an
non-existing operator
- // coordinator, then respond with an exception to the call. If task
and coordinator exist,
- // then we assume that the call from the TaskManager was valid, and
any bubbling exception
- // needs to cause a job failure.
-
- final Execution exec =
executionGraph.getRegisteredExecutions().get(taskExecutionId);
- if (exec == null || exec.getState() != ExecutionState.RUNNING) {
- // This situation is common when cancellation happens, or when the
task failed while the
- // event was just being dispatched asynchronously on the TM side.
- // It should be fine in those expected situations to just ignore
this event, but, to be
- // on the safe, we notify the TM that the event could not be
delivered.
- throw new TaskNotRunningException(
- "Task is not known or in state running on the
JobManager.");
- }
-
- final OperatorCoordinatorHolder coordinator =
coordinatorMap.get(operatorId);
- if (coordinator == null) {
- throw new FlinkException("No coordinator registered for operator "
+ operatorId);
- }
-
- try {
-
coordinator.handleEventFromOperator(exec.getParallelSubtaskIndex(), evt);
- } catch (Throwable t) {
- ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
- globalFailureHandler.accept(t);
- }
- }
-
- public CompletableFuture<CoordinationResponse>
deliverCoordinationRequestToCoordinator(
- OperatorID operator, CoordinationRequest request) throws
FlinkException {
-
- final OperatorCoordinatorHolder coordinatorHolder =
coordinatorMap.get(operator);
- if (coordinatorHolder == null) {
- throw new FlinkException("Coordinator of operator " + operator + "
does not exist");
- }
-
- final OperatorCoordinator coordinator =
coordinatorHolder.coordinator();
- if (coordinator instanceof CoordinationRequestHandler) {
- return ((CoordinationRequestHandler)
coordinator).handleCoordinationRequest(request);
- } else {
- throw new FlinkException(
- "Coordinator of operator " + operator + " cannot handle
client event");
- }
- }
+ CompletableFuture<CoordinationResponse>
deliverCoordinationRequestToCoordinator(
+ OperatorID operator, CoordinationRequest request) throws
FlinkException;
Review comment:
JavaDocs are missing for this interface.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/StateWithExecutionGraphTest.java
##########
@@ -75,34 +82,112 @@ public void
testSuspendCanBeCalledWhenExecutionGraphHasReachedGloballyTerminalSt
}
}
+ @Test
+ public void testOperatorCoordinatorShutdownOnLeave() throws Exception {
+ try (MockStateWithExecutionGraphContext context =
+ new MockStateWithExecutionGraphContext()) {
+
+ final TestingOperatorCoordinatorHandler
testingOperatorCoordinatorHandler =
+ new TestingOperatorCoordinatorHandler();
+ final TestingStateWithExecutionGraph stateWithExecutionGraph =
+ createStateWithExecutionGraph(context,
testingOperatorCoordinatorHandler);
+
+
stateWithExecutionGraph.onLeave(AdaptiveSchedulerTest.DummyState.class);
+
+ assertThat(testingOperatorCoordinatorHandler.isRunning(),
is(false));
+ }
+ }
+
+ @Test
+ public void testSuspendToFinished() throws Exception {
+ try (MockStateWithExecutionGraphContext context =
+ new MockStateWithExecutionGraphContext()) {
+
+ final TestingStateWithExecutionGraph stateWithExecutionGraph =
+ createStateWithExecutionGraph(context);
+
+ context.setExpectFinished(aeg -> assertThat(aeg.getState(),
is(JobStatus.SUSPENDED)));
+
+ stateWithExecutionGraph.suspend(new RuntimeException());
+ }
+ }
+
+ @Test
+ public void testOnGloballyTerminalStateCalled() throws Exception {
+ MockStateWithExecutionGraphContext context = new
MockStateWithExecutionGraphContext();
+
+ StateTrackingMockExecutionGraph mockExecutionGraph = new
StateTrackingMockExecutionGraph();
+ final TestingStateWithExecutionGraph stateWithExecutionGraph =
+ createStateWithExecutionGraph(context, mockExecutionGraph);
+
+ mockExecutionGraph.completeTerminationFuture(JobStatus.FINISHED);
+
+ context.close();
+
+ assertThat(
+ stateWithExecutionGraph.getGloballyTerminalStateFuture().get(),
+ is(JobStatus.FINISHED));
+ }
+
+ @Test
+ public void
testOnGloballyTerminalStateNotCalledOnNonGloballyTerminalState() throws
Exception {
+ MockStateWithExecutionGraphContext context = new
MockStateWithExecutionGraphContext();
+
+ StateTrackingMockExecutionGraph mockExecutionGraph = new
StateTrackingMockExecutionGraph();
+ final TestingStateWithExecutionGraph stateWithExecutionGraph =
+ createStateWithExecutionGraph(context, mockExecutionGraph);
+
+ mockExecutionGraph.completeTerminationFuture(JobStatus.SUSPENDED);
+
+ context.close();
+
+
assertThat(stateWithExecutionGraph.getGloballyTerminalStateFuture().isDone(),
is(false));
+ }
+
+ private TestingStateWithExecutionGraph createStateWithExecutionGraph(
+ MockStateWithExecutionGraphContext context) {
+ final ExecutionGraph executionGraph = new
StateTrackingMockExecutionGraph();
+ return createStateWithExecutionGraph(context, executionGraph);
+ }
+
+ private TestingStateWithExecutionGraph createStateWithExecutionGraph(
+ MockStateWithExecutionGraphContext context,
+ OperatorCoordinatorHandler operatorCoordinatorHandler) {
+ final ExecutionGraph executionGraph = new
StateTrackingMockExecutionGraph();
+ return createStateWithExecutionGraph(context, executionGraph,
operatorCoordinatorHandler);
+ }
+
+ private TestingStateWithExecutionGraph createStateWithExecutionGraph(
+ MockStateWithExecutionGraphContext context, ExecutionGraph
executionGraph) {
+ final OperatorCoordinatorHandler operatorCoordinatorHandler =
+ new OperatorCoordinatorHandlerImplementation(
+ executionGraph,
+ globalFailure -> {
+ throw new FlinkRuntimeException(
+ "No global failures are expected",
globalFailure);
+ });
Review comment:
Could we use the `TestingOperatorCoordinatorHandler` here?
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java
##########
@@ -689,7 +690,8 @@ public Logger getLogger() {
}
}
- private static class MockOperatorCoordinatorHandler extends
OperatorCoordinatorHandler {
+ private static class MockOperatorCoordinatorHandler
+ extends OperatorCoordinatorHandlerImplementation {
Review comment:
Do we need to extend the implementation or could we add a
`TestingOperatorCoordinatorHandler`?
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/OperatorCoordinatorHandlerImplementation.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.scheduler;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+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.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import
org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.TaskNotRunningException;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+/** Handler implementation for the {@link OperatorCoordinator
OperatorCoordinators}. */
+public class OperatorCoordinatorHandlerImplementation implements
OperatorCoordinatorHandler {
Review comment:
The naming pattern is unconventional. I think we do have the pattern
`DefaultXYZ` or `XYZImpl` in the code base but not `XYZImplementation`. Ideally
one picks a meaningful name which tells us about the specific implementation
but if you don't have a good idea, then stick to the existing naming patterns.
--
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]