From 042aa4172045f4b8b8788e0c20edc8ddbd720da7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 14 Apr 2021 15:38:49 +0200 Subject: [PATCH 01/10] [hotfix][coordination] Add Main-Thread check to OperatorEvent sending on Execution. Now that the threading model of the OperatorCoordinatorHolder has been adjusted, we can finally have this check and use this method in accordance with the general contract of the Scheduler. --- .../java/org/apache/flink/runtime/executiongraph/Execution.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index a39000acb00fd..c336d235b7da9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -854,6 +854,8 @@ private void triggerCheckpointHelper( */ public CompletableFuture sendOperatorEvent( OperatorID operatorId, SerializedValue event) { + + assertRunningInJobMasterMainThread(); final LogicalSlot slot = assignedResource; if (slot != null && (getState() == RUNNING || getState() == INITIALIZING)) { From 1c3e57e2a2355b63116a587e5df1180bdc1e2d3f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 11 Apr 2021 18:55:14 +0200 Subject: [PATCH 02/10] [FLINK-18071][coordination] (part 2) OperatorCoordinatorHolder does not implement OperatorCoordinator interface any more Originally it was designed that the OperatorCoordinatorHolder has the same interface as the OperatorCoordinator and simply adds some hooks around the checkpoint triggering procedure. However, the OperatorCoordinatorHolder is becoming the glue between the scheduler threads and the scheduler's view on tasks and their status, and the OperatorCoordinator threads and their simplified view on the execution state. This means they do require different interfaces. --- .../operators/coordination/OperatorCoordinatorHolder.java | 5 +---- .../org/apache/flink/runtime/scheduler/DefaultScheduler.java | 5 +++-- .../org/apache/flink/runtime/scheduler/SchedulerBase.java | 2 +- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java index 21974c361f476..4901f6e5d357d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java @@ -114,7 +114,7 @@ * an event) are also enqueued back into the scheduler main-thread executor, strictly in order. */ public class OperatorCoordinatorHolder - implements OperatorCoordinator, OperatorCoordinatorCheckpointContext { + implements OperatorCoordinatorCheckpointContext, AutoCloseable { private final OperatorCoordinator coordinator; private final OperatorID operatorId; @@ -179,7 +179,6 @@ public int currentParallelism() { // OperatorCoordinator Interface // ------------------------------------------------------------------------ - @Override public void start() throws Exception { mainThreadExecutor.assertRunningInMainThread(); checkState(context.isInitialized(), "Coordinator Context is not yet initialized"); @@ -192,13 +191,11 @@ public void close() throws Exception { context.unInitialize(); } - @Override public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception { mainThreadExecutor.assertRunningInMainThread(); coordinator.handleEventFromOperator(subtask, event); } - @Override public void subtaskFailed(int subtask, @Nullable Throwable reason) { mainThreadExecutor.assertRunningInMainThread(); coordinator.subtaskFailed(subtask, reason); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 2618d276b7314..33e86361e8267 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -43,7 +43,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory; @@ -550,7 +550,8 @@ private void notifyCoordinatorOfCancellation(ExecutionVertex vertex) { return; } - for (OperatorCoordinator coordinator : vertex.getJobVertex().getOperatorCoordinators()) { + for (OperatorCoordinatorHolder coordinator : + vertex.getJobVertex().getOperatorCoordinators()) { coordinator.subtaskFailed(vertex.getParallelSubtaskIndex(), null); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 50d335054ea47..0ab567224c36a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -451,7 +451,7 @@ private void notifyCoordinatorsOfSubtaskRestore( private void notifyCoordinatorsOfEmptyGlobalRestore() throws Exception { for (final ExecutionJobVertex ejv : getExecutionGraph().getAllVertices().values()) { - for (final OperatorCoordinator coordinator : ejv.getOperatorCoordinators()) { + for (final OperatorCoordinatorHolder coordinator : ejv.getOperatorCoordinators()) { coordinator.resetToCheckpoint(OperatorCoordinator.NO_CHECKPOINT, null); } } From 3881410865145027be7545499e869639444f85ff Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 11 Apr 2021 19:27:01 +0200 Subject: [PATCH 03/10] [FLINK-18071][coordination] (part 3) Adjust OperatorEventValve to accept self-contained "send actions". The "send actions" (as Callables) contain the events and versioned target information, so they can precisely address the recipient regardless of whether they are sent immediately or later, after the target task has failed/recovered. --- .../coordination/OperatorEventValve.java | 43 +++++++++++++------ 1 file changed, 31 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java index 99eb7aaea1247..25cc5191e48a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; @@ -30,6 +31,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.function.BiFunction; @@ -105,15 +107,25 @@ public void sendEvent( CompletableFuture result) { checkRunsInMainThread(); + final Callable> sendAction = + () -> eventSender.apply(event, subtask); + sendEvent(sendAction, subtask, result); + } + + public void sendEvent( + Callable> sendAction, + int subtask, + CompletableFuture result) { + checkRunsInMainThread(); + if (!shut) { - final CompletableFuture ack = eventSender.apply(event, subtask); - FutureUtils.forward(ack, result); + callSendAction(sendAction, result); return; } final List eventsForTask = blockedEvents.computeIfAbsent(subtask, (key) -> new ArrayList<>()); - eventsForTask.add(new BlockedEvent(event, subtask, result)); + eventsForTask.add(new BlockedEvent(sendAction, result)); } /** @@ -190,9 +202,7 @@ public void openValveAndUnmarkCheckpoint() { for (List eventsForTask : blockedEvents.values()) { for (BlockedEvent blockedEvent : eventsForTask) { - final CompletableFuture ackFuture = - eventSender.apply(blockedEvent.event, blockedEvent.subtask); - FutureUtils.forward(ackFuture, blockedEvent.future); + callSendAction(blockedEvent.sendAction, blockedEvent.future); } } blockedEvents.clear(); @@ -242,21 +252,30 @@ private void checkRunsInMainThread() { } } + private static void callSendAction( + Callable> sendAction, + CompletableFuture result) { + try { + final CompletableFuture sendResult = sendAction.call(); + FutureUtils.forward(sendResult, result); + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); + result.completeExceptionally(t); + } + } + // ------------------------------------------------------------------------ private static final class BlockedEvent { - final SerializedValue event; + final Callable> sendAction; final CompletableFuture future; - final int subtask; BlockedEvent( - SerializedValue event, - int subtask, + Callable> sendAction, CompletableFuture future) { - this.event = event; + this.sendAction = sendAction; this.future = future; - this.subtask = subtask; } } } From 5b87970b76a02354c37e7b38a05af338742ee1b5 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 11 Apr 2021 20:05:47 +0200 Subject: [PATCH 04/10] [FLINK-18071][coordination] (part 4) Add to Execution a future for states INITIALIZING/RUNNING This doubles as a listener for when the execution has reached the state INITIALIZING or RUNNING. --- .../runtime/executiongraph/Execution.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index c336d235b7da9..6ce4a4191098a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -149,6 +149,13 @@ public class Execution private final CompletableFuture taskManagerLocationFuture; + /** + * Gets completed successfully when the task switched to {@link ExecutionState#INITIALIZING} or + * {@link ExecutionState#RUNNING}. If the task never switches to those state, but fails + * immediately, then this future never completes. + */ + private final CompletableFuture initializingOrRunningFuture; + private volatile ExecutionState state = CREATED; private LogicalSlot assignedResource; @@ -214,6 +221,7 @@ public Execution( this.terminalStateFuture = new CompletableFuture<>(); this.releaseFuture = new CompletableFuture<>(); this.taskManagerLocationFuture = new CompletableFuture<>(); + this.initializingOrRunningFuture = new CompletableFuture<>(); this.assignedResource = null; } @@ -352,6 +360,23 @@ public void setInitialState(@Nullable JobManagerTaskRestore taskRestore) { this.taskRestore = taskRestore; } + /** + * Gets a future that completes once the task execution reaches one of the states {@link + * ExecutionState#INITIALIZING} or {@link ExecutionState#RUNNING}. If this task never reaches + * these states (for example because the task is cancelled before it was properly deployed and + * restored), then this future will never complete. + * + *

The future is completed already in the {@link ExecutionState#INITIALIZING} state, because + * various running actions are already possible in that state (the task already accepts and + * sends events and network data for task recovery). (Note that in earlier versions, the + * INITIALIZING state was not separate but part of the RUNNING state). + * + *

This future is always completed from the job master's main thread. + */ + public CompletableFuture getInitializingOrRunningFuture() { + return initializingOrRunningFuture; + } + /** * Gets a future that completes once the task execution reaches a terminal state. The future * will be completed with specific state that the execution reached. This future is always @@ -1428,7 +1453,9 @@ private boolean transitionState( } } - if (targetState.isTerminal()) { + if (targetState == INITIALIZING || targetState == RUNNING) { + initializingOrRunningFuture.complete(null); + } else if (targetState.isTerminal()) { // complete the terminal state future terminalStateFuture.complete(targetState); } From ecfe15c0d57eacd7e357696b3415db90f551716a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 12 Apr 2021 12:37:31 +0200 Subject: [PATCH 05/10] [FLINK-18071][coordination] (part 6) Communication from Coordinators to Tasks happens through gateways that are scoped to a single execution attempt. That way the events have a well specified target, no matter how much the actual sending is delayed due to race conditions. This also un-ignores the CoordinatorEventsExactlyOnceITCase that now runs stable with this fix. --- .../coordination/ExecutionSubtaskAccess.java | 118 +++++++ .../coordination/OperatorCoordinator.java | 82 ++++- .../OperatorCoordinatorHolder.java | 173 ++++++----- .../coordination/OperatorEventValve.java | 97 +----- .../RecreateOnResetOperatorCoordinator.java | 17 +- .../operators/coordination/SubtaskAccess.java | 87 ++++++ .../coordination/SubtaskGatewayImpl.java | 84 +++++ .../coordination/TaskNotRunningException.java | 5 + .../util/IncompleteFuturesTracker.java | 103 ++++++ .../source/coordinator/SourceCoordinator.java | 11 + .../coordinator/SourceCoordinatorContext.java | 54 +++- .../CoordinatorEventsExactlyOnceITCase.java | 238 +++++++++++--- .../coordination/EventReceivingTasks.java | 223 +++++++++++++ .../coordination/MockOperatorCoordinator.java | 5 + .../MockOperatorCoordinatorContext.java | 53 +--- .../OperatorCoordinatorHolderTest.java | 191 +++++------- .../OperatorCoordinatorSchedulerTest.java | 27 +- .../coordination/OperatorEventValveTest.java | 104 ++----- ...ecreateOnResetOperatorCoordinatorTest.java | 38 ++- .../coordination/TestEventSender.java | 109 ------- .../TestingOperatorCoordinator.java | 21 +- .../util/IncompleteFuturesTrackerTest.java | 105 +++++++ .../SourceCoordinatorContextTest.java | 97 +++--- .../coordinator/SourceCoordinatorTest.java | 293 +++++++----------- .../SourceCoordinatorTestBase.java | 117 +++++-- .../coordinator/TestingSplitEnumerator.java | 255 +++++++++++++++ .../CollectSinkOperatorCoordinator.java | 5 + 27 files changed, 1864 insertions(+), 848 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestEventSender.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/TestingSplitEnumerator.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java new file mode 100644 index 0000000000000..fc363dec88e49 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java @@ -0,0 +1,118 @@ +/* + * 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.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; + +/** + * An implementation of the {@link SubtaskAccess} interface that uses the ExecutionGraph's classes, + * specifically {@link Execution} and {@link ExecutionJobVertex} to access tasks. + */ +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 + // without this, the futures would only fail after the RPC system hits the ask-timeout. + taskExecution + .getTerminalStateFuture() + .thenAccept( + (state) -> + futuresTracker.failAllFutures( + new FlinkException("Task is no longer running"))); + } + + @Override + public Callable> createEventSendAction( + SerializedValue event) { + return () -> { + final CompletableFuture result = + taskExecution.sendOperatorEvent(operator, event); + futuresTracker.trackFutureWhileIncomplete(result); + return result; + }; + } + + @Override + public int getSubtaskIndex() { + return taskExecution.getParallelSubtaskIndex(); + } + + @Override + public ExecutionAttemptID currentAttempt() { + return taskExecution.getAttemptId(); + } + + @Override + public CompletableFuture hasSwitchedToRunning() { + return taskExecution.getInitializingOrRunningFuture(); + } + + @Override + public boolean isStillRunning() { + return taskExecution.getState() == ExecutionState.RUNNING + || taskExecution.getState() == ExecutionState.INITIALIZING; + } + + // ------------------------------------------------------------------------ + + static final class ExecutionJobVertexSubtaskAccess implements SubtaskAccessFactory { + + private final ExecutionJobVertex ejv; + private final OperatorID operator; + + ExecutionJobVertexSubtaskAccess(ExecutionJobVertex ejv, OperatorID operator) { + this.ejv = checkNotNull(ejv); + this.operator = checkNotNull(operator); + } + + @Override + public SubtaskAccess getAccessForSubtask(int subtask) { + if (subtask < 0 || subtask >= ejv.getParallelism()) { + throw new IllegalArgumentException( + "Subtask index out of bounds [0, " + ejv.getParallelism() + ')'); + } + + final Execution taskExecution = + ejv.getTaskVertices()[subtask].getCurrentExecutionAttempt(); + return new ExecutionSubtaskAccess(taskExecution, operator); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java index 4f50c8cd850ba..24a546c02f3dd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators.coordination; import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.messages.Acknowledge; @@ -44,6 +45,30 @@ *

Coordinators that involve more complex operations should hence spawn threads to handle the I/O * work. The methods on the {@link Context} are safe to be called from another thread than the * thread that calls the Coordinator's methods. + * + *

Consistency

+ * + *

The coordinator's view of the task execution is highly simplified, compared to the Scheduler's + * view, but allows for consistent interaction with the operators running on the parallel subtasks. + * In particular, the following methods are guaranteed to be called strictly in order: + * + *

    + *
  1. {@link #subtaskReady(int, SubtaskGateway)}: Called once you can send events to the subtask. + * The provided gateway is bound to that specific task. This is the start of interaction with + * the operator subtasks. + *
  2. {@link #subtaskFailed(int, Throwable)}: Called for each subtask as soon as the subtask + * execution failed or was cancelled. At this point, interaction with the subtask should stop. + *
  3. {@link #subtaskReset(int, long)} or {@link #resetToCheckpoint(long, byte[])}: Once the + * scheduler determined which checkpoint to restore, these methods notify the coordinator of + * that. The former method is called in case of a regional failure/recovery (affecting + * possible a subset of subtasks), the later method in case of a global failure/recovery. This + * method should be used to determine which actions to recover, because it tells you with + * checkpoint to fall back to. The coordinator implementation needs to recover the + * interactions with the relevant tasks since the checkpoint that is restored. + *
  4. {@link #subtaskReady(int, SubtaskGateway)}: Called again, once the recovered tasks are + * ready to go. This is later than {@link #subtaskReset(int, long)}, because between those + * methods, the task are scheduled and deployed. + *
*/ public interface OperatorCoordinator extends CheckpointListener, AutoCloseable { @@ -74,7 +99,8 @@ public interface OperatorCoordinator extends CheckpointListener, AutoCloseable { // ------------------------------------------------------------------------ /** - * Hands an OperatorEvent from a task (on the Task Manager) to this coordinator. + * Hands an OperatorEvent coming from a parallel Operator instances (one of the parallel + * subtasks). * * @throws Exception Any exception thrown by this method results in a full job failure and * recovery. @@ -106,6 +132,9 @@ public interface OperatorCoordinator extends CheckpointListener, AutoCloseable { *
  • Every event sent after the checkpoint future is completed is considered to be after the * checkpoint. * + * + * @throws Exception Any exception thrown by this method results in a full job failure and * + * recovery. */ void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) throws Exception; @@ -148,7 +177,7 @@ default void notifyCheckpointAborted(long checkpointId) {} * the coordinator. * * - * In both cases, the coordinator should reset to an empty (new) state. + *

    In both cases, the coordinator should reset to an empty (new) state. * *

    Restoring implicitly notifies of Checkpoint Completion

    * @@ -183,6 +212,17 @@ default void notifyCheckpointAborted(long checkpointId) {} */ void subtaskReset(int subtask, long checkpointId); + /** + * This is called when a subtask of the Operator becomes ready to receive events, both after + * initial startup and after task failover. The given {@code SubtaskGateway} can be used to send + * events to the executed subtask. + * + *

    The given {@code SubtaskGateway} is bound to that specific execution attempt that became + * ready. All events sent through the gateway target that execution attempt; if the attempt is + * no longer running by the time the event is sent, then the events are failed. + */ + void subtaskReady(int subtask, SubtaskGateway gateway); + // ------------------------------------------------------------------------ // ------------------------------------------------------------------------ @@ -195,15 +235,6 @@ interface Context { /** Gets the ID of the operator to which the coordinator belongs. */ OperatorID getOperatorId(); - /** - * Sends an event to the parallel subtask with the given subtask index. - * - *

    The returned future is completed successfully once the event has been received by the - * target TaskManager. The future is completed exceptionally if the event cannot be sent. - * That includes situations where the target task is not running. - */ - CompletableFuture sendEvent(OperatorEvent evt, int targetSubtask); - /** * Fails the job and trigger a global failover operation. * @@ -225,6 +256,35 @@ interface Context { // ------------------------------------------------------------------------ + /** + * The {@code SubtaskGateway} is the way to interact with a specific parallel instance of the + * Operator (an Operator subtask), like sending events to the operator. + */ + interface SubtaskGateway { + + /** + * Sends an event to the parallel subtask with the given subtask index. + * + *

    The returned future is completed successfully once the event has been received by the + * target TaskManager. The future is completed exceptionally if the event cannot be sent. + * That includes situations where the target task is not running. + */ + CompletableFuture sendEvent(OperatorEvent evt); + + /** + * Gets the execution attempt for the subtask execution attempt that this gateway + * communicates with. + */ + ExecutionAttemptID getExecution(); + + /** + * Gets the subtask index of the parallel operator instance this gateway communicates with. + */ + int getSubtask(); + } + + // ------------------------------------------------------------------------ + /** * The provider creates an OperatorCoordinator and takes a {@link Context} to pass to the * OperatorCoordinator. This method is, for example, called on the job manager when the diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java index 4901f6e5d357d..89e1fc49c8b71 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java @@ -21,13 +21,11 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.checkpoint.OperatorCoordinatorCheckpointContext; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; -import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.concurrent.FutureUtils; 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.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.TemporaryClassLoaderContext; @@ -36,10 +34,8 @@ import javax.annotation.Nullable; -import java.io.IOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.function.BiFunction; import java.util.function.Consumer; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -119,6 +115,7 @@ public class OperatorCoordinatorHolder private final OperatorCoordinator coordinator; private final OperatorID operatorId; private final LazyInitializedCoordinatorContext context; + private final SubtaskAccess.SubtaskAccessFactory taskAccesses; private final OperatorEventValve eventValve; private final int operatorParallelism; @@ -131,25 +128,31 @@ private OperatorCoordinatorHolder( final OperatorID operatorId, final OperatorCoordinator coordinator, final LazyInitializedCoordinatorContext context, - final OperatorEventValve eventValve, + final SubtaskAccess.SubtaskAccessFactory taskAccesses, final int operatorParallelism, final int operatorMaxParallelism) { this.operatorId = checkNotNull(operatorId); this.coordinator = checkNotNull(coordinator); this.context = checkNotNull(context); - this.eventValve = checkNotNull(eventValve); + this.taskAccesses = checkNotNull(taskAccesses); this.operatorParallelism = operatorParallelism; this.operatorMaxParallelism = operatorMaxParallelism; + + this.eventValve = new OperatorEventValve(); } public void lazyInitialize( Consumer globalFailureHandler, ComponentMainThreadExecutor mainThreadExecutor) { + this.globalFailureHandler = globalFailureHandler; this.mainThreadExecutor = mainThreadExecutor; + eventValve.setMainThreadExecutorForValidation(mainThreadExecutor); context.lazyInitialize(globalFailureHandler, mainThreadExecutor); + + setupAllSubtaskGateways(); } // ------------------------------------------------------------------------ @@ -199,12 +202,16 @@ public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exc public void subtaskFailed(int subtask, @Nullable Throwable reason) { mainThreadExecutor.assertRunningInMainThread(); coordinator.subtaskFailed(subtask, reason); - eventValve.resetForTask(subtask); } @Override public void subtaskReset(int subtask, long checkpointId) { mainThreadExecutor.assertRunningInMainThread(); + + // this needs to happen first, so that the coordinator may access the gateway + // in the 'subtaskReset()' function (even though they cannot send events, yet). + setupSubtaskGateway(subtask); + coordinator.subtaskReset(subtask, checkpointId); } @@ -244,10 +251,22 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData mainThreadExecutor.assertRunningInMainThread(); } - eventValve.reset(); + eventValve.openValveAndUnmarkCheckpoint(); + if (context != null) { context.resetFailed(); } + + // when initial savepoints are restored, this call comes before the mainThreadExecutor + // is available, which is needed to set up these gateways. So during the initial restore, + // we ignore this, and instead the gateways are set up in the "lazyInitialize" method, which + // is called when the scheduler is properly set up. + // this is a bit clumsy, but it is caused by the non-straightforward initialization of the + // ExecutionGraph and Scheduler. + if (mainThreadExecutor != null) { + setupAllSubtaskGateways(); + } + coordinator.resetToCheckpoint(checkpointId, checkpointData); } @@ -257,20 +276,21 @@ private void checkpointCoordinatorInternal( final CompletableFuture coordinatorCheckpoint = new CompletableFuture<>(); - coordinatorCheckpoint.whenCompleteAsync( - (success, failure) -> { - if (failure != null) { - result.completeExceptionally(failure); - } else { - try { - eventValve.shutValve(checkpointId); - result.complete(success); - } catch (Exception e) { - result.completeExceptionally(e); - } - } - }, - mainThreadExecutor); + coordinatorCheckpoint.whenComplete( + (success, failure) -> + mainThreadExecutor.execute( + () -> { + if (failure != null) { + result.completeExceptionally(failure); + } else { + try { + eventValve.shutValve(checkpointId); + result.complete(success); + } catch (Exception e) { + result.completeExceptionally(e); + } + } + })); try { eventValve.markForCheckpoint(checkpointId); @@ -304,6 +324,51 @@ public void abortCurrentTriggering() { mainThreadExecutor.execute(eventValve::openValveAndUnmarkCheckpoint); } + // ------------------------------------------------------------------------ + // miscellaneous helpers + // ------------------------------------------------------------------------ + + private void setupAllSubtaskGateways() { + for (int i = 0; i < operatorParallelism; i++) { + setupSubtaskGateway(i); + } + } + + private void setupSubtaskGateway(int subtask) { + // this gets an access to the latest task execution attempt. + final SubtaskAccess sta = taskAccesses.getAccessForSubtask(subtask); + + final OperatorCoordinator.SubtaskGateway gateway = + new SubtaskGatewayImpl(sta, eventValve, mainThreadExecutor); + + // We need to do this synchronously here, otherwise we violate the contract that + // 'subtaskFailed()' will never overtake 'subtaskReady()'. + // An alternative, if we ever figure out that this cannot work synchronously here, + // is that we re-enqueue all actions (like 'subtaskFailed()' and 'subtaskRestored()') + // back into the main thread executor, rather than directly calling the OperatorCoordinator + FutureUtils.assertNoException( + sta.hasSwitchedToRunning() + .thenAccept( + (ignored) -> { + mainThreadExecutor.assertRunningInMainThread(); + + // this is a guard in case someone accidentally makes the + // notification asynchronous + assert sta.isStillRunning(); + + notifySubtaskReady(subtask, gateway); + })); + } + + private void notifySubtaskReady(int subtask, OperatorCoordinator.SubtaskGateway gateway) { + try { + coordinator.subtaskReady(subtask, gateway); + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + globalFailureHandler.accept(new FlinkException("Error from OperatorCoordinator", t)); + } + } + // ------------------------------------------------------------------------ // Factories // ------------------------------------------------------------------------ @@ -319,24 +384,17 @@ public static OperatorCoordinatorHolder create( serializedProvider.deserializeValue(classLoader); final OperatorID opId = provider.getOperatorId(); - final BiFunction< - SerializedValue, Integer, CompletableFuture> - eventSender = - (serializedEvent, subtask) -> { - final Execution executionAttempt = - jobVertex.getTaskVertices()[subtask] - .getCurrentExecutionAttempt(); - return executionAttempt.sendOperatorEvent(opId, serializedEvent); - }; + final SubtaskAccess.SubtaskAccessFactory taskAccesses = + new ExecutionSubtaskAccess.ExecutionJobVertexSubtaskAccess(jobVertex, opId); return create( opId, provider, - eventSender, jobVertex.getName(), jobVertex.getGraph().getUserClassLoader(), jobVertex.getParallelism(), - jobVertex.getMaxParallelism()); + jobVertex.getMaxParallelism(), + taskAccesses); } } @@ -344,25 +402,26 @@ public static OperatorCoordinatorHolder create( static OperatorCoordinatorHolder create( final OperatorID opId, final OperatorCoordinator.Provider coordinatorProvider, - final BiFunction< - SerializedValue, Integer, CompletableFuture> - eventSender, final String operatorName, final ClassLoader userCodeClassLoader, final int operatorParallelism, - final int operatorMaxParallelism) + final int operatorMaxParallelism, + final SubtaskAccess.SubtaskAccessFactory taskAccesses) throws Exception { - final OperatorEventValve valve = new OperatorEventValve(eventSender); - final LazyInitializedCoordinatorContext context = new LazyInitializedCoordinatorContext( - opId, valve, operatorName, userCodeClassLoader, operatorParallelism); + opId, operatorName, userCodeClassLoader, operatorParallelism); final OperatorCoordinator coordinator = coordinatorProvider.create(context); return new OperatorCoordinatorHolder( - opId, coordinator, context, valve, operatorParallelism, operatorMaxParallelism); + opId, + coordinator, + context, + taskAccesses, + operatorParallelism, + operatorMaxParallelism); } // ------------------------------------------------------------------------ @@ -385,7 +444,6 @@ private static final class LazyInitializedCoordinatorContext LoggerFactory.getLogger(LazyInitializedCoordinatorContext.class); private final OperatorID operatorId; - private final OperatorEventValve eventValve; private final String operatorName; private final ClassLoader userCodeClassLoader; private final int operatorParallelism; @@ -397,12 +455,10 @@ private static final class LazyInitializedCoordinatorContext public LazyInitializedCoordinatorContext( final OperatorID operatorId, - final OperatorEventValve eventValve, final String operatorName, final ClassLoader userCodeClassLoader, final int operatorParallelism) { this.operatorId = checkNotNull(operatorId); - this.eventValve = checkNotNull(eventValve); this.operatorName = checkNotNull(operatorName); this.userCodeClassLoader = checkNotNull(userCodeClassLoader); this.operatorParallelism = operatorParallelism; @@ -435,33 +491,6 @@ public OperatorID getOperatorId() { return operatorId; } - @Override - public CompletableFuture sendEvent( - final OperatorEvent evt, final int targetSubtask) { - checkInitialized(); - - if (targetSubtask < 0 || targetSubtask >= currentParallelism()) { - throw new IllegalArgumentException( - String.format( - "subtask index %d out of bounds [0, %d).", - targetSubtask, currentParallelism())); - } - - final SerializedValue serializedEvent; - try { - serializedEvent = new SerializedValue<>(evt); - } catch (IOException e) { - // we do not expect that this exception is handled by the caller, so we make it - // unchecked so that it can bubble up - throw new FlinkRuntimeException("Cannot serialize operator event", e); - } - - final CompletableFuture result = new CompletableFuture<>(); - schedulerExecutor.execute( - () -> eventValve.sendEvent(serializedEvent, targetSubtask, result)); - return result; - } - @Override public void failJob(final Throwable cause) { checkInitialized(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java index 25cc5191e48a5..98c90d5499617 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java @@ -22,25 +22,19 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.SerializedValue; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; -import java.util.function.BiFunction; /** * The event value is the connection through which operator events are sent, from coordinator to - * operator.It can temporarily block events from going through, buffering them, and releasing them - * later. - * - *

    The valve can also drop buffered events for all or selected targets. + * operator. It can temporarily block 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. * *

    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 @@ -50,11 +44,7 @@ final class OperatorEventValve { private static final long NO_CHECKPOINT = Long.MIN_VALUE; - private final BiFunction< - SerializedValue, Integer, CompletableFuture> - eventSender; - - private final Map> blockedEvents = new LinkedHashMap<>(); + private final List blockedEvents = new ArrayList<>(); private long currentCheckpointId; @@ -64,15 +54,8 @@ final class OperatorEventValve { @Nullable private ComponentMainThreadExecutor mainThreadExecutor; - /** - * Constructs a new OperatorEventValve, passing the events to the given function when the valve - * is open or opened again. The second parameter of the BiFunction is the target operator - * subtask index. - */ - public OperatorEventValve( - BiFunction, Integer, CompletableFuture> - eventSender) { - this.eventSender = eventSender; + /** Constructs a new OperatorEventValve. */ + public OperatorEventValve() { this.currentCheckpointId = NO_CHECKPOINT; this.lastCheckpointId = Long.MIN_VALUE; } @@ -94,38 +77,21 @@ public boolean isShut() { * future. * *

    If the valve is closed this buffers the event and returns an incomplete future. The future - * is completed with the original result once the valve is opened. If the event is never sent - * (because it gets dropped through a call to {@link #reset()} or {@link #resetForTask(int)}, - * then the returned future till be completed exceptionally. + * is completed with the original result once the valve is opened again. * *

    This method makes no assumptions and gives no guarantees from which thread the result * future gets completed. */ - public void sendEvent( - SerializedValue event, - int subtask, - CompletableFuture result) { - checkRunsInMainThread(); - - final Callable> sendAction = - () -> eventSender.apply(event, subtask); - sendEvent(sendAction, subtask, result); - } - public void sendEvent( Callable> sendAction, - int subtask, CompletableFuture result) { checkRunsInMainThread(); - if (!shut) { + if (shut) { + blockedEvents.add(new BlockedEvent(sendAction, result)); + } else { callSendAction(sendAction, result); - return; } - - final List eventsForTask = - blockedEvents.computeIfAbsent(subtask, (key) -> new ArrayList<>()); - eventsForTask.add(new BlockedEvent(sendAction, result)); } /** @@ -200,52 +166,13 @@ public void openValveAndUnmarkCheckpoint() { return; } - for (List eventsForTask : blockedEvents.values()) { - for (BlockedEvent blockedEvent : eventsForTask) { - callSendAction(blockedEvent.sendAction, blockedEvent.future); - } + for (BlockedEvent blockedEvent : blockedEvents) { + callSendAction(blockedEvent.sendAction, blockedEvent.future); } blockedEvents.clear(); shut = false; } - /** Drops all blocked events for a specific subtask. */ - public void resetForTask(int subtask) { - checkRunsInMainThread(); - - final List events = blockedEvents.remove(subtask); - failAllFutures(events); - } - - /** Resets the valve, dropping all blocked events and opening the valve. */ - public void reset() { - checkRunsInMainThread(); - - final List events = new ArrayList<>(); - for (List taskEvents : blockedEvents.values()) { - if (taskEvents != null) { - events.addAll(taskEvents); - } - } - blockedEvents.clear(); - shut = false; - currentCheckpointId = NO_CHECKPOINT; - - failAllFutures(events); - } - - private static void failAllFutures(@Nullable List events) { - if (events == null || events.isEmpty()) { - return; - } - - final Exception failureCause = - new FlinkException("Event discarded due to failure of target task"); - for (BlockedEvent evt : events) { - evt.future.completeExceptionally(failureCause); - } - } - private void checkRunsInMainThread() { if (mainThreadExecutor != null) { mainThreadExecutor.assertRunningInMainThread(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinator.java index 5b12388bcad39..0d3d5f8b586bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinator.java @@ -20,7 +20,6 @@ Licensed to the Apache Software Foundation (ASF) under one import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.ThrowingConsumer; import org.apache.flink.util.function.ThrowingRunnable; @@ -93,6 +92,11 @@ public void subtaskReset(int subtask, long checkpointId) { coordinator.applyCall("subtaskReset", c -> c.subtaskReset(subtask, checkpointId)); } + @Override + public void subtaskReady(int subtask, SubtaskGateway gateway) { + coordinator.applyCall("subtaskReady", c -> c.subtaskReady(subtask, gateway)); + } + @Override public void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) throws Exception { @@ -158,6 +162,7 @@ void waitForAllAsyncCallsFinish() throws Exception { // --------------------- + /** The provider for a private RecreateOnResetOperatorCoordinator. */ public abstract static class Provider implements OperatorCoordinator.Provider { private static final long serialVersionUID = 3002837631612629071L; private final OperatorID operatorID; @@ -210,16 +215,6 @@ public OperatorID getOperatorId() { return context.getOperatorId(); } - @Override - public synchronized CompletableFuture sendEvent( - OperatorEvent evt, int targetSubtask) { - // Do not enter the sending procedure if the context has been quiesced. - if (quiesced) { - return CompletableFuture.completedFuture(Acknowledge.get()); - } - return context.sendEvent(evt, targetSubtask); - } - @Override public synchronized void failJob(Throwable cause) { if (quiesced) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java new file mode 100644 index 0000000000000..c654af860f321 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java @@ -0,0 +1,87 @@ +/* + * 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.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.SerializedValue; + +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; + +/** + * This interface offers access to a parallel subtask in the scope of the subtask as the target for + * sending {@link OperatorEvent}s from an {@link OperatorCoordinator}. + * + *

    Important: An instance of this access must be bound to one specific execution attempt + * of the subtask. After that execution attempt failed, that instance must not bind to another + * execution attempt, but a new instance would need to be created via the {@link + * SubtaskAccess.SubtaskAccessFactory}. + */ +interface SubtaskAccess { + + /** + * Creates a Callable that, when invoked, sends the event to the execution attempt of the + * subtask that this {@code SubtaskAccess} instance binds to. The resulting future from the + * sending is returned by the callable. + * + *

    This let's the caller target the specific subtask without necessarily sending the event + * now (for example, the event may be sent at a later point due to checkpoint alignment through + * the {@link OperatorEventValve}). + */ + Callable> createEventSendAction( + SerializedValue event); + + /** Gets the parallel subtask index of the target subtask. */ + int getSubtaskIndex(); + + /** Gets the execution attempt ID of the attempt that this instance is bound to. */ + ExecutionAttemptID currentAttempt(); + + /** + * The future returned here completes once the target subtask is in a running state. As running + * state classify the states {@link ExecutionState#RUNNING} and {@link + * ExecutionState#INITIALIZING}. + */ + CompletableFuture hasSwitchedToRunning(); + + /** + * Checks whether the execution is still in a running state. See {@link #hasSwitchedToRunning()} + * for details. + */ + boolean isStillRunning(); + + // ------------------------------------------------------------------------ + + /** + * While the {@link SubtaskAccess} is bound to an execution attempt of a subtask (like an {@link + * org.apache.flink.runtime.executiongraph.Execution}, this factory is bound to the operator as + * a whole (like in the scope of an {@link + * org.apache.flink.runtime.executiongraph.ExecutionJobVertex}. + */ + interface SubtaskAccessFactory { + + /** + * Creates an access to the current execution attempt of the subtask with the given + * subtaskIndex. + */ + SubtaskAccess getAccessForSubtask(int subtaskIndex); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java new file mode 100644 index 0000000000000..0e8dade4e5310 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java @@ -0,0 +1,84 @@ +/* + * 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.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.SerializedValue; + +import java.io.IOException; +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}. + */ +class SubtaskGatewayImpl implements OperatorCoordinator.SubtaskGateway { + + private final SubtaskAccess subtaskAccess; + private final OperatorEventValve valve; + private final Executor sendingExecutor; + + SubtaskGatewayImpl( + SubtaskAccess subtaskAccess, OperatorEventValve valve, Executor sendingExecutor) { + this.subtaskAccess = subtaskAccess; + this.valve = valve; + this.sendingExecutor = sendingExecutor; + } + + @Override + public CompletableFuture sendEvent(OperatorEvent evt) { + if (!isReady()) { + throw new FlinkRuntimeException("SubtaskGateway is not ready, task not yet running."); + } + + final SerializedValue serializedEvent; + try { + serializedEvent = new SerializedValue<>(evt); + } catch (IOException e) { + // we do not expect that this exception is handled by the caller, so we make it + // unchecked so that it can bubble up + throw new FlinkRuntimeException("Cannot serialize operator event", e); + } + + final Callable> sendAction = + subtaskAccess.createEventSendAction(serializedEvent); + + final CompletableFuture result = new CompletableFuture<>(); + sendingExecutor.execute(() -> valve.sendEvent(sendAction, result)); + return result; + } + + @Override + public ExecutionAttemptID getExecution() { + return subtaskAccess.currentAttempt(); + } + + @Override + public int getSubtask() { + return subtaskAccess.getSubtaskIndex(); + } + + private boolean isReady() { + return subtaskAccess.hasSwitchedToRunning().isDone(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/TaskNotRunningException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/TaskNotRunningException.java index 2f5a60cf2739c..3b6c11f9b5f71 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/TaskNotRunningException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/TaskNotRunningException.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators.coordination; +import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.util.FlinkException; /** An exception indicating that a target task is not running. */ @@ -28,4 +29,8 @@ public class TaskNotRunningException extends FlinkException { public TaskNotRunningException(String message) { super(message); } + + public TaskNotRunningException(String taskDescription, ExecutionState state) { + super(taskDescription + " is currently not RUNNING but in state " + state); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java new file mode 100644 index 0000000000000..9904ce0008e25 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java @@ -0,0 +1,103 @@ +/* + * 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.util; + +import org.apache.flink.annotation.VisibleForTesting; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.locks.ReentrantLock; + +/** + * This tracker remembers CompletableFutures as long as they are incomplete and allows us to fail + * them later. + */ +public final class IncompleteFuturesTracker { + + private final ReentrantLock lock = new ReentrantLock(); + + private final HashSet> incompleteFutures = new HashSet<>(); + + @Nullable private Throwable failureCause; + + public void trackFutureWhileIncomplete(CompletableFuture future) { + // this is only a best-effort shortcut for efficiency + if (future.isDone()) { + return; + } + + lock.lock(); + try { + if (failureCause != null) { + future.completeExceptionally(failureCause); + return; + } + + incompleteFutures.add(future); + } finally { + lock.unlock(); + } + + // when the future completes, it removes itself from the set + future.whenComplete((success, failure) -> removeFromSet(future)); + } + + public void failAllFutures(Throwable cause) { + final Collection> futuresToFail; + + lock.lock(); + try { + if (failureCause != null) { + // already failed + return; + } + failureCause = cause; + futuresToFail = new ArrayList<>(incompleteFutures); + incompleteFutures.clear(); + } finally { + lock.unlock(); + } + + // we do the actual failing outside the lock scope for efficiency, because there + // may be synchronous actions triggered on the future + for (CompletableFuture future : futuresToFail) { + future.completeExceptionally(cause); + } + } + + void removeFromSet(CompletableFuture future) { + lock.lock(); + try { + incompleteFutures.remove(future); + } finally { + lock.unlock(); + } + } + + @VisibleForTesting + Set> getTrackedFutures() { + return Collections.unmodifiableSet(incompleteFutures); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java index 2cb02353f3654..0c9852ccd7c03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinator.java @@ -186,6 +186,7 @@ public void subtaskFailed(int subtaskId, @Nullable Throwable reason) { subtaskId, operatorName); context.unregisterSourceReader(subtaskId); + context.subtaskNotReady(subtaskId); }, "handling subtask %d failure", subtaskId); @@ -214,6 +215,16 @@ public void subtaskReset(int subtaskId, long checkpointId) { checkpointId); } + @Override + public void subtaskReady(int subtask, SubtaskGateway gateway) { + assert subtask == gateway.getSubtask(); + + runInEventLoop( + () -> context.subtaskReady(gateway), + "making event gateway to subtask %d available", + subtask); + } + @Override public void checkpointCoordinator(long checkpointId, CompletableFuture result) { runInEventLoop( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java index 598bc86ed4e81..d5b1d20d2c25b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java @@ -90,6 +90,7 @@ public class SourceCoordinatorContext private final SplitAssignmentTracker assignmentTracker; private final SourceCoordinatorProvider.CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private final OperatorCoordinator.SubtaskGateway[] subtaskGateways; private final String coordinatorThreadName; private volatile boolean closed; @@ -127,6 +128,9 @@ public SourceCoordinatorContext( this.registeredReaders = new ConcurrentHashMap<>(); this.assignmentTracker = splitAssignmentTracker; this.coordinatorThreadName = coordinatorThreadFactory.getCoordinatorThreadName(); + this.subtaskGateways = + new OperatorCoordinator.SubtaskGateway + [operatorCoordinatorContext.currentParallelism()]; final Executor errorHandlingCoordinatorExecutor = (runnable) -> @@ -144,9 +148,13 @@ public MetricGroup metricGroup() { @Override public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + checkSubtaskIndex(subtaskId); + callInCoordinatorThread( () -> { - operatorCoordinatorContext.sendEvent(new SourceEventWrapper(event), subtaskId); + final OperatorCoordinator.SubtaskGateway gateway = + getGatewayAndCheckReady(subtaskId); + gateway.sendEvent(new SourceEventWrapper(event)); return null; }, String.format("Failed to send event %s to subtask %d", event, subtaskId)); @@ -182,6 +190,9 @@ public void assignSplits(SplitsAssignment assignment) { .assignment() .forEach( (id, splits) -> { + final OperatorCoordinator.SubtaskGateway gateway = + getGatewayAndCheckReady(id); + final AddSplitEvent addSplitEvent; try { addSplitEvent = @@ -190,7 +201,8 @@ public void assignSplits(SplitsAssignment assignment) { throw new FlinkRuntimeException( "Failed to serialize splits.", e); } - operatorCoordinatorContext.sendEvent(addSplitEvent, id); + + gateway.sendEvent(addSplitEvent); }); return null; }, @@ -199,10 +211,14 @@ public void assignSplits(SplitsAssignment assignment) { @Override public void signalNoMoreSplits(int subtask) { + checkSubtaskIndex(subtask); + // Ensure the split assignment is done by the the coordinator executor. callInCoordinatorThread( () -> { - operatorCoordinatorContext.sendEvent(new NoMoreSplitsEvent(), subtask); + final OperatorCoordinator.SubtaskGateway gateway = + getGatewayAndCheckReady(subtask); + gateway.sendEvent(new NoMoreSplitsEvent()); return null; // void return value }, "Failed to send 'NoMoreSplits' to reader " + subtask); @@ -237,6 +253,29 @@ public void close() throws InterruptedException { // --------- Package private additional methods for the SourceCoordinator ------------ + void subtaskReady(OperatorCoordinator.SubtaskGateway gateway) { + final int subtask = gateway.getSubtask(); + if (subtaskGateways[subtask] == null) { + subtaskGateways[gateway.getSubtask()] = gateway; + } else { + throw new IllegalStateException("Already have a subtask gateway for " + subtask); + } + } + + void subtaskNotReady(int subtaskIndex) { + subtaskGateways[subtaskIndex] = null; + } + + OperatorCoordinator.SubtaskGateway getGatewayAndCheckReady(int subtaskIndex) { + final OperatorCoordinator.SubtaskGateway gateway = subtaskGateways[subtaskIndex]; + if (gateway != null) { + return gateway; + } + + throw new IllegalStateException( + String.format("Subtask %d is not ready yet to receive events.", subtaskIndex)); + } + /** * Fail the job with the given cause. * @@ -318,6 +357,15 @@ OperatorCoordinator.Context getCoordinatorContext() { // ---------------- private helper methods ----------------- + private void checkSubtaskIndex(int subtaskIndex) { + if (subtaskIndex < 0 || subtaskIndex >= getCoordinatorContext().currentParallelism()) { + throw new IllegalArgumentException( + String.format( + "Subtask index %d is out of bounds [0, %s)", + subtaskIndex, getCoordinatorContext().currentParallelism())); + } + } + /** * A helper method that delegates the callable to the coordinator thread if the current thread * is not the coordinator thread, otherwise call the callable right away. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/CoordinatorEventsExactlyOnceITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/CoordinatorEventsExactlyOnceITCase.java index e227278c5783e..33222c349418b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/CoordinatorEventsExactlyOnceITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/CoordinatorEventsExactlyOnceITCase.java @@ -46,6 +46,8 @@ import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; +import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.SerializedValue; @@ -55,7 +57,6 @@ import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import javax.annotation.Nullable; @@ -63,29 +64,63 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Random; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkState; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** * Integration Test case that validates the exactly-once mechanism for coordinator events around - * checkpoints. + * checkpoints. The test checks for two distinct problems related to exactly-once event delivery: * - *

    The test provokes the corner cases of the mechanism described in {@link - * OperatorCoordinatorHolder}. + *

    1. Delayed events

    + * + *

    When the OperatorCoordinator runs in its own thread (which they commonly do), it is possible + * that races occur between when an event is meant to be sent, and when it actually gets sent, and + * the notifications about task failures. + * + *

    For example, an event that was meant to target task-execution-attempt X might actually get + * sent when task-execution-attempt X+1 is already running. If the coordinator has not yet processed + * the information that task-execution-attempt X is no longer running, and that + * task-execution-attempt X+1 is now running, then we don't want events to sneakily reach + * task-execution-attempt X+1. Otherwise we cannot reason about which events need to be resent + * because of the failure, and which do not. + * + *

    So this test checks the following condition: After a task has failed over to a new execution, + * events being sent must not reach that new task before the notification has reached the + * coordinator about the previous task failure and the new task execution. + * + *

    2. Exactly-once alignment between multiple Coordinators

    + * + *

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

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

    In the example below, the events {@code c & d} must be held back until after the barrier + * injection. * *

      * Coordinator one events: => a . . b . |trigger| . . |complete| . . c . . d . |barrier| . e . f
    @@ -102,8 +137,9 @@ public class CoordinatorEventsExactlyOnceITCase extends TestLogger {
     
         private static final ConfigOption ACC_NAME =
                 ConfigOptions.key("acc").stringType().noDefaultValue();
    -    private static final String OPERATOR_1_ACCUMULATOR = "op-acc-1";
    -    private static final String OPERATOR_2_ACCUMULATOR = "op-acc-2";
    +
    +    private static final String OPERATOR_1_NAME = "operator-1";
    +    private static final String OPERATOR_2_NAME = "operator-2";
     
         private static MiniCluster miniCluster;
     
    @@ -131,29 +167,26 @@ public static void shutdownMiniCluster() throws Exception {
         // ------------------------------------------------------------------------
     
         @Test
    -    @Ignore
         public void test() throws Exception {
             final int numEvents1 = 200;
             final int numEvents2 = 5;
             final int delay1 = 1;
             final int delay2 = 200;
     
    -        final JobVertex task1 =
    -                buildJobVertex("TASK_1", numEvents1, delay1, OPERATOR_1_ACCUMULATOR);
    -        final JobVertex task2 =
    -                buildJobVertex("TASK_2", numEvents2, delay2, OPERATOR_2_ACCUMULATOR);
    +        final JobVertex task1 = buildJobVertex(OPERATOR_1_NAME, numEvents1, delay1);
    +        final JobVertex task2 = buildJobVertex(OPERATOR_2_NAME, numEvents2, delay2);
     
             final JobGraph jobGraph =
                     JobGraphBuilder.newStreamingJobGraphBuilder()
                             .setJobName("Coordinator Events Job")
                             .addJobVertices(Arrays.asList(task1, task2))
    -                        .setJobCheckpointingSettings(createCheckpointSettings(task1, task2))
    +                        .setJobCheckpointingSettings(createCheckpointSettings())
                             .build();
     
             final JobExecutionResult result = miniCluster.executeJobBlocking(jobGraph);
     
    -        checkListContainsSequence(result.getAccumulatorResult(OPERATOR_2_ACCUMULATOR), numEvents2);
    -        checkListContainsSequence(result.getAccumulatorResult(OPERATOR_1_ACCUMULATOR), numEvents1);
    +        checkListContainsSequence(result.getAccumulatorResult(OPERATOR_1_NAME), numEvents1);
    +        checkListContainsSequence(result.getAccumulatorResult(OPERATOR_2_NAME), numEvents2);
         }
     
         private static void checkListContainsSequence(List ints, int length) {
    @@ -170,21 +203,24 @@ private static void checkListContainsSequence(List ints, int length) {
         }
     
         private static void failList(List ints, int length) {
    -        fail("List did not contain expected sequence of " + length + " elements, but was: " + ints);
    +        fail(
    +                String.format(
    +                        "List did not contain expected sequence of %d elements, but was: (%d elements): %s",
    +                        length, ints.size(), ints));
         }
     
         // ------------------------------------------------------------------------
         //  test setup helpers
         // ------------------------------------------------------------------------
     
    -    private static JobVertex buildJobVertex(String name, int numEvents, int delay, String accName)
    +    private static JobVertex buildJobVertex(String name, int numEvents, int delay)
                 throws IOException {
             final JobVertex vertex = new JobVertex(name);
             final OperatorID opId = OperatorID.fromJobVertexID(vertex.getID());
     
             vertex.setParallelism(1);
             vertex.setInvokableClass(EventCollectingTask.class);
    -        vertex.getConfiguration().setString(ACC_NAME, accName);
    +        vertex.getConfiguration().setString(ACC_NAME, name);
     
             final OperatorCoordinator.Provider provider =
                     new OperatorCoordinator.Provider() {
    @@ -196,7 +232,7 @@ public OperatorID getOperatorId() {
     
                         @Override
                         public OperatorCoordinator create(OperatorCoordinator.Context context) {
    -                        return new EventSendingCoordinator(context, numEvents, delay);
    +                        return new EventSendingCoordinator(context, name, numEvents, delay);
                         }
                     };
     
    @@ -205,7 +241,7 @@ public OperatorCoordinator create(OperatorCoordinator.Context context) {
             return vertex;
         }
     
    -    private static JobCheckpointingSettings createCheckpointSettings(JobVertex... vertices) {
    +    private static JobCheckpointingSettings createCheckpointSettings() {
             final CheckpointCoordinatorConfiguration coordCfg =
                     new CheckpointCoordinatorConfiguration.CheckpointCoordinatorConfigurationBuilder()
                             .setMaxConcurrentCheckpoints(1)
    @@ -240,31 +276,55 @@ public String toString() {
     
         // ------------------------------------------------------------------------
     
    -    private static final class EventSendingCoordinator implements OperatorCoordinator, Runnable {
    +    /**
    +     * The coordinator that sends events and completes checkpoints.
    +     *
    +     * 

    All consistency guaranteed for the coordinator apply to order or method invocations (like + * {@link #subtaskFailed(int, Throwable)}, {@link #subtaskReset(int, long)} or {@link + * #checkpointCoordinator(long, CompletableFuture)}) and the order in which actions are done + * (sending events and completing checkpoints). Tho consistently evaluate this, but with + * concurrency against the scheduler thread that calls this coordinator implements a simple + * mailbox that moves the method handling into a separate thread, but keeps the order. + */ + private static final class EventSendingCoordinator implements OperatorCoordinator { private final Context context; - private ScheduledExecutorService executor; - private volatile ScheduledFuture periodicTask; + private final ExecutorService mailboxExecutor; + private final ScheduledExecutorService scheduledExecutor; private final int delay; private final int maxNumber; private int nextNumber; - private volatile CompletableFuture requestedCheckpoint; + private CompletableFuture requestedCheckpoint; private CompletableFuture nextToComplete; private final int failAtMessage; private boolean failedBefore; - private EventSendingCoordinator(Context context, int numEvents, int delay) { + private final ArrayDeque> recoveredTaskRunning = new ArrayDeque<>(); + + private SubtaskGateway subtaskGateway; + private boolean workLoopRunning; + + private EventSendingCoordinator(Context context, String name, int numEvents, int delay) { checkArgument(delay > 0); checkArgument(numEvents >= 3); this.context = context; this.maxNumber = numEvents; this.delay = delay; - this.executor = Executors.newSingleThreadScheduledExecutor(); + this.mailboxExecutor = + Executors.newSingleThreadExecutor( + new DispatcherThreadFactory( + Thread.currentThread().getThreadGroup(), + "Coordinator Mailbox for " + name)); + this.scheduledExecutor = + Executors.newSingleThreadScheduledExecutor( + new DispatcherThreadFactory( + Thread.currentThread().getThreadGroup(), + "Coordinator Periodic Actions for " + name)); this.failAtMessage = numEvents / 3 + new Random().nextInt(numEvents / 3); } @@ -274,8 +334,11 @@ public void start() throws Exception {} @Override public void close() throws Exception { - executor.shutdownNow(); - executor.awaitTermination(10, TimeUnit.SECONDS); + scheduledExecutor.shutdownNow(); + assertTrue(scheduledExecutor.awaitTermination(10, TimeUnit.MINUTES)); + + mailboxExecutor.shutdownNow(); + assertTrue(mailboxExecutor.awaitTermination(10, TimeUnit.MINUTES)); } @Override @@ -285,41 +348,130 @@ public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exc String.format("Don't recognize event '%s' from task %d.", event, subtask)); } - if (periodicTask != null) { - throw new Exception("periodic already running"); + synchronized (recoveredTaskRunning) { + // signal the previous task that its recovered task is now running + final CompletableFuture prevTaskFuture = recoveredTaskRunning.peekLast(); + if (prevTaskFuture != null) { + prevTaskFuture.complete(null); + } + // add a future for this task + recoveredTaskRunning.addLast(new CompletableFuture<>()); } - periodicTask = - executor.scheduleWithFixedDelay(this, delay, delay, TimeUnit.MILLISECONDS); + + // first, we hand this over to the mailbox thread, so we preserve order on operations, + // even if the action is only to do a thread safe scheduling into the scheduledExecutor + runInMailbox( + () -> { + checkState(!workLoopRunning); + checkState(subtaskGateway != null); + + workLoopRunning = true; + scheduleSingleAction(); + }); } @Override public void subtaskFailed(int subtask, @Nullable Throwable reason) { - periodicTask.cancel(false); - periodicTask = null; - executor.execute(() -> nextNumber = 0); + // simulate a heavy thread race here: the mailbox has a last enqueued action before the + // cancellation is processed. But through a race, the mailbox freezes for a while and in + // that time, the task already went through a recovery cycle. By the time the mailbox + // unfreezes, the new task will be the recipient of new events. + // to simulate this race, we wait precisely until the point when the new task pings the + // coordinator before unfreezing the mailbox + runInMailbox( + () -> { + final CompletableFuture thisTasksFuture; + synchronized (recoveredTaskRunning) { + thisTasksFuture = recoveredTaskRunning.peekFirst(); + } + + if (thisTasksFuture != null) { + try { + thisTasksFuture.get(); + } catch (Exception ignored) { + } + + synchronized (recoveredTaskRunning) { + recoveredTaskRunning.removeFirst(); + } + } + + executeSingleAction(); + }); + + // after the late racing action, this is the proper shutdown + runInMailbox( + () -> { + workLoopRunning = false; + subtaskGateway = null; + }); } @Override public void subtaskReset(int subtask, long checkpointId) {} + @Override + public void subtaskReady(int subtask, SubtaskGateway gateway) { + runInMailbox( + () -> { + checkState(!workLoopRunning); + subtaskGateway = gateway; + }); + } + @Override public void resetToCheckpoint( final long checkpointId, @Nullable final byte[] checkpointData) throws Exception { - executor.execute(() -> nextNumber = bytesToInt(checkpointData)); + runInMailbox( + () -> nextNumber = checkpointData == null ? 0 : bytesToInt(checkpointData)); } @Override public void checkpointCoordinator(long checkpointId, CompletableFuture result) throws Exception { - requestedCheckpoint = result; + runInMailbox(() -> requestedCheckpoint = result); } @Override public void notifyCheckpointComplete(long checkpointId) {} + void runInMailbox(Runnable action) { + mailboxExecutor.execute( + () -> { + try { + action.run(); + } catch (Throwable t) { + // this eventually kills the test, which is harsh but the simplest way + // to make sure exceptions that bubble up are not swallowed and hide + // problems. To simplify debugging, we print the stack trace here before + // the exception + t.printStackTrace(); + ExceptionUtils.rethrow(t); + } + }); + } + + void scheduleSingleAction() { + try { + scheduledExecutor.schedule( + () -> runInMailbox(this::executeSingleAction), + delay, + TimeUnit.MILLISECONDS); + } catch (RejectedExecutionException e) { + if (!scheduledExecutor.isShutdown()) { + throw e; + } + } + } + @SuppressWarnings("CallToPrintStackTrace") - @Override - public void run() { + private void executeSingleAction() { + if (!workLoopRunning) { + // if the delay scheduler put a task in here, but we really aren't + // working any more, then skip this + return; + } + try { handleCheckpoint(); sendNextEvent(); @@ -330,6 +482,9 @@ public void run() { t.printStackTrace(); System.exit(-1); } + + // schedule the next step + scheduleSingleAction(); } private void handleCheckpoint() { @@ -351,10 +506,11 @@ private void sendNextEvent() { } if (nextNumber == maxNumber) { - context.sendEvent(new EndEvent(), 0); + subtaskGateway.sendEvent(new EndEvent()); } else { - context.sendEvent(new IntegerEvent(nextNumber), 0); + subtaskGateway.sendEvent(new IntegerEvent(nextNumber)); } + nextNumber++; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java new file mode 100644 index 0000000000000..baee4b10c20e3 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java @@ -0,0 +1,223 @@ +/* + * 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.Executors; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.SerializedValue; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * A test implementation of the BiFunction interface used as the underlying event sender in the + * {@link OperatorCoordinatorHolder}. + */ +public class EventReceivingTasks implements SubtaskAccess.SubtaskAccessFactory { + + public static EventReceivingTasks createForNotYetRunningTasks() { + return new EventReceivingTasks(false, null); + } + + public static EventReceivingTasks createForRunningTasks() { + return new EventReceivingTasks(true, null); + } + + public static EventReceivingTasks createForRunningTasksFailingRpcs(Throwable rpcException) { + return new EventReceivingTasks(true, rpcException); + } + + // ------------------------------------------------------------------------ + + final ArrayList events = new ArrayList<>(); + + @Nullable private final Throwable eventSendingFailureCause; + + private final Map subtasks = new HashMap<>(); + + private final boolean createdTasksAreRunning; + + private EventReceivingTasks( + final boolean createdTasksAreRunning, + @Nullable final Throwable eventSendingFailureCause) { + this.createdTasksAreRunning = createdTasksAreRunning; + this.eventSendingFailureCause = eventSendingFailureCause; + } + + // ------------------------------------------------------------------------ + // Access to sent events + // ------------------------------------------------------------------------ + + public int getNumberOfSentEvents() { + return events.size(); + } + + public List getAllSentEvents() { + return events; + } + + public List getSentEventsForSubtask(int subtaskIndex) { + return events.stream() + .filter((evt) -> evt.subtask == subtaskIndex) + .map((evt) -> evt.event) + .collect(Collectors.toList()); + } + + // ------------------------------------------------------------------------ + // Controlling the life cycle of the target tasks + // ------------------------------------------------------------------------ + + @Override + public SubtaskAccess getAccessForSubtask(int subtask) { + return subtasks.computeIfAbsent( + subtask, (subtaskIdx) -> new TestSubtaskAccess(subtaskIdx, createdTasksAreRunning)); + } + + public OperatorCoordinator.SubtaskGateway createGatewayForSubtask(int subtask) { + final SubtaskAccess sta = getAccessForSubtask(subtask); + return new SubtaskGatewayImpl(sta, new OperatorEventValve(), Executors.directExecutor()); + } + + public void switchTaskToRunning(int subtask) { + final TestSubtaskAccess task = subtasks.get(subtask); + if (task != null) { + task.switchToRunning(); + } else { + throw new IllegalArgumentException("No subtask created for " + subtask); + } + } + + public void switchAllTasksToRunning() { + for (TestSubtaskAccess tsa : subtasks.values()) { + tsa.switchToRunning(); + } + } + + Callable> createSendAction(OperatorEvent event, int subtask) { + return () -> { + events.add(new EventWithSubtask(event, subtask)); + + return eventSendingFailureCause == null + ? CompletableFuture.completedFuture(Acknowledge.get()) + : FutureUtils.completedExceptionally(eventSendingFailureCause); + }; + } + + // ------------------------------------------------------------------------ + + /** A combination of an {@link OperatorEvent} and the target subtask it is sent to. */ + public static final class EventWithSubtask { + + public final OperatorEvent event; + public final int subtask; + + public EventWithSubtask(OperatorEvent event, int subtask) { + this.event = event; + this.subtask = subtask; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final EventWithSubtask that = (EventWithSubtask) o; + return subtask == that.subtask && event.equals(that.event); + } + + @Override + public int hashCode() { + return Objects.hash(event, subtask); + } + + @Override + public String toString() { + return event + " => subtask " + subtask; + } + } + + // ------------------------------------------------------------------------ + + private final class TestSubtaskAccess implements SubtaskAccess { + + private final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID(); + private final CompletableFuture running; + private final int subtaskIndex; + + private TestSubtaskAccess(int subtaskIndex, boolean isRunning) { + this.subtaskIndex = subtaskIndex; + this.running = new CompletableFuture<>(); + if (isRunning) { + switchToRunning(); + } + } + + @Override + public Callable> createEventSendAction( + SerializedValue event) { + + final OperatorEvent deserializedEvent; + try { + deserializedEvent = event.deserializeValue(getClass().getClassLoader()); + } catch (IOException | ClassNotFoundException e) { + throw new AssertionError(e); + } + + return createSendAction(deserializedEvent, subtaskIndex); + } + + @Override + public int getSubtaskIndex() { + return subtaskIndex; + } + + @Override + public ExecutionAttemptID currentAttempt() { + return executionAttemptId; + } + + @Override + public CompletableFuture hasSwitchedToRunning() { + return running; + } + + @Override + public boolean isStillRunning() { + return true; + } + + void switchToRunning() { + running.complete(null); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java index 73c2b905b8e4c..bb1f84a38c15b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java @@ -53,6 +53,11 @@ public void subtaskReset(int subtask, long checkpointId) { throw new UnsupportedOperationException(); } + @Override + public void subtaskReady(int subtask, SubtaskGateway gateway) { + throw new UnsupportedOperationException(); + } + @Override public void checkpointCoordinator(long checkpointId, CompletableFuture result) { throw new UnsupportedOperationException(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinatorContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinatorContext.java index c20915c45a43b..a1e1dfe18f352 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinatorContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinatorContext.java @@ -19,53 +19,31 @@ Licensed to the Apache Software Foundation (ASF) under one package org.apache.flink.runtime.operators.coordination; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.util.FlinkRuntimeException; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; +/** A simple implementation of {@link OperatorCoordinator.Context} for testing purposes. */ public class MockOperatorCoordinatorContext implements OperatorCoordinator.Context { + private final OperatorID operatorID; private final ClassLoader userCodeClassLoader; private final int numSubtasks; - private final boolean failEventSending; - private final Map> eventsToOperator; private boolean jobFailed; private Throwable jobFailureReason; public MockOperatorCoordinatorContext(OperatorID operatorID, int numSubtasks) { - this(operatorID, numSubtasks, true); - } - - public MockOperatorCoordinatorContext( - OperatorID operatorID, int numSubtasks, boolean failEventSending) { - this( - operatorID, - numSubtasks, - failEventSending, - MockOperatorCoordinatorContext.class.getClassLoader()); + this(operatorID, numSubtasks, MockOperatorCoordinatorContext.class.getClassLoader()); } public MockOperatorCoordinatorContext(OperatorID operatorID, ClassLoader userCodeClassLoader) { - this(operatorID, 1, true, userCodeClassLoader); + this(operatorID, 1, userCodeClassLoader); } public MockOperatorCoordinatorContext( - OperatorID operatorID, - int numSubtasks, - boolean failEventSending, - ClassLoader userCodeClassLoader) { + OperatorID operatorID, int numSubtasks, ClassLoader userCodeClassLoader) { this.operatorID = operatorID; this.numSubtasks = numSubtasks; - this.eventsToOperator = new HashMap<>(); this.jobFailed = false; this.jobFailureReason = null; - this.failEventSending = failEventSending; this.userCodeClassLoader = userCodeClassLoader; } @@ -74,19 +52,6 @@ public OperatorID getOperatorId() { return operatorID; } - @Override - public CompletableFuture sendEvent(OperatorEvent evt, int targetSubtask) { - eventsToOperator.computeIfAbsent(targetSubtask, subtaskId -> new ArrayList<>()).add(evt); - if (failEventSending) { - CompletableFuture future = new CompletableFuture<>(); - future.completeExceptionally( - new FlinkRuntimeException("Testing Exception to fail event sending.")); - return future; - } else { - return CompletableFuture.completedFuture(Acknowledge.get()); - } - } - @Override public void failJob(Throwable cause) { jobFailed = true; @@ -105,14 +70,6 @@ public ClassLoader getUserCodeClassloader() { // ------------------------------- - public List getEventsToOperatorBySubtaskId(int subtaskId) { - return eventsToOperator.get(subtaskId); - } - - public Map> getEventsToOperator() { - return eventsToOperator; - } - public boolean isJobFailed() { return jobFailed; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java index 442a5c08bce1b..4eacd16d4a44f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java @@ -22,10 +22,8 @@ import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutorService; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.operators.coordination.TestEventSender.EventWithSubtask; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks.EventWithSubtask; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -40,7 +38,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; @@ -74,9 +71,9 @@ public void checkNoGlobalFailure() throws Exception { @Test public void checkpointFutureInitiallyNotDone() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); final CompletableFuture checkpointFuture = new CompletableFuture<>(); holder.checkpointCoordinator(1L, checkpointFuture); @@ -86,9 +83,9 @@ public void checkpointFutureInitiallyNotDone() throws Exception { @Test public void completedCheckpointFuture() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); final byte[] testData = new byte[] {11, 22, 33, 44}; @@ -102,101 +99,72 @@ public void completedCheckpointFuture() throws Exception { @Test public void eventsBeforeCheckpointFutureCompletionPassThrough() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); holder.checkpointCoordinator(1L, new CompletableFuture<>()); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1), 1); + getCoordinator(holder).getSubtaskGateway(1).sendEvent(new TestOperatorEvent(1)); - assertThat(sender.events, contains(new EventWithSubtask(new TestOperatorEvent(1), 1))); + assertThat(tasks.getSentEventsForSubtask(1), contains(new TestOperatorEvent(1))); } @Test public void eventsAreBlockedAfterCheckpointFutureCompletes() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); triggerAndCompleteCheckpoint(holder, 10L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1337), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(1337)); - assertTrue(sender.events.isEmpty()); + assertEquals(0, tasks.getNumberOfSentEvents()); } @Test public void abortedCheckpointReleasesBlockedEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); triggerAndCompleteCheckpoint(holder, 123L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1337), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(1337)); holder.abortCurrentTriggering(); - assertThat(sender.events, contains(new EventWithSubtask(new TestOperatorEvent(1337), 0))); + assertThat(tasks.getSentEventsForSubtask(0), contains(new TestOperatorEvent(1337))); } @Test public void sourceBarrierInjectionReleasesBlockedEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); triggerAndCompleteCheckpoint(holder, 1111L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1337), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(1337)); holder.afterSourceBarrierInjection(1111L); - assertThat(sender.events, contains(new EventWithSubtask(new TestOperatorEvent(1337), 0))); - } - - @Test - public void failedTasksDropsBlockedEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); - - triggerAndCompleteCheckpoint(holder, 1000L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(0), 0); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1), 1); - holder.subtaskFailed(1, null); - holder.abortCurrentTriggering(); - - assertThat(sender.events, contains(new EventWithSubtask(new TestOperatorEvent(0), 0))); + assertThat(tasks.getSentEventsForSubtask(0), contains(new TestOperatorEvent(1337))); } @Test public void restoreOpensValveEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); triggerAndCompleteCheckpoint(holder, 1000L); holder.resetToCheckpoint(1L, new byte[0]); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(999), 1); - - assertThat(sender.events, contains(new EventWithSubtask(new TestOperatorEvent(999), 1))); - } - - @Test - public void restoreDropsBlockedEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + getCoordinator(holder).getSubtaskGateway(1).sendEvent(new TestOperatorEvent(999)); - triggerAndCompleteCheckpoint(holder, 1000L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(0), 0); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1), 1); - holder.resetToCheckpoint(2L, new byte[0]); - - assertTrue(sender.events.isEmpty()); + assertThat(tasks.getSentEventsForSubtask(1), contains(new TestOperatorEvent(999))); } @Test public void lateCompleteCheckpointFutureDoesNotBlockEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); final CompletableFuture holderFuture = new CompletableFuture<>(); holder.checkpointCoordinator(1000L, holderFuture); @@ -210,16 +178,16 @@ public void lateCompleteCheckpointFutureDoesNotBlockEvents() throws Exception { future1.complete(new byte[0]); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(123), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(123)); - assertThat(sender.events, contains(new EventWithSubtask(new TestOperatorEvent(123), 0))); + assertThat(tasks.events, contains(new EventWithSubtask(new TestOperatorEvent(123), 0))); } @Test public void triggeringFailsIfOtherTriggeringInProgress() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); holder.checkpointCoordinator(11L, new CompletableFuture<>()); @@ -233,56 +201,56 @@ public void triggeringFailsIfOtherTriggeringInProgress() throws Exception { @Test public void takeCheckpointAfterSuccessfulCheckpoint() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(0), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(0)); triggerAndCompleteCheckpoint(holder, 22L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(1)); holder.afterSourceBarrierInjection(22L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(2), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(2)); triggerAndCompleteCheckpoint(holder, 23L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(3), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(3)); holder.afterSourceBarrierInjection(23L); assertThat( - sender.events, + tasks.getSentEventsForSubtask(0), contains( - new EventWithSubtask(new TestOperatorEvent(0), 0), - new EventWithSubtask(new TestOperatorEvent(1), 0), - new EventWithSubtask(new TestOperatorEvent(2), 0), - new EventWithSubtask(new TestOperatorEvent(3), 0))); + new TestOperatorEvent(0), + new TestOperatorEvent(1), + new TestOperatorEvent(2), + new TestOperatorEvent(3))); } @Test public void takeCheckpointAfterAbortedCheckpoint() throws Exception { - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, TestingOperatorCoordinator::new); + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(0), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(0)); triggerAndCompleteCheckpoint(holder, 22L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(1), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(1)); holder.abortCurrentTriggering(); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(2), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(2)); triggerAndCompleteCheckpoint(holder, 23L); - getCoordinator(holder).getContext().sendEvent(new TestOperatorEvent(3), 0); + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(3)); holder.afterSourceBarrierInjection(23L); assertThat( - sender.events, + tasks.getSentEventsForSubtask(0), contains( - new EventWithSubtask(new TestOperatorEvent(0), 0), - new EventWithSubtask(new TestOperatorEvent(1), 0), - new EventWithSubtask(new TestOperatorEvent(2), 0), - new EventWithSubtask(new TestOperatorEvent(3), 0))); + new TestOperatorEvent(0), + new TestOperatorEvent(1), + new TestOperatorEvent(2), + new TestOperatorEvent(3))); } @Test @@ -295,9 +263,9 @@ public void handleEventFromOperator(int subtask, OperatorEvent event) { context.failJob(new RuntimeException("Artificial Exception")); } }; - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks tasks = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = - createCoordinatorHolder(sender, coordinatorProvider); + createCoordinatorHolder(tasks, coordinatorProvider); holder.handleEventFromOperator(0, new TestOperatorEvent()); assertNotNull(globalFailure); @@ -355,7 +323,7 @@ private void checkpointEventValueAtomicity( new ComponentMainThreadExecutorServiceAdapter( (ScheduledExecutorService) executor, Thread.currentThread()); - final TestEventSender sender = new TestEventSender(); + final EventReceivingTasks sender = EventReceivingTasks.createForRunningTasks(); final OperatorCoordinatorHolder holder = createCoordinatorHolder(sender, coordinatorCtor, mainThreadExecutor); @@ -377,9 +345,10 @@ private void checkpointEventValueAtomicity( assertTrue(checkpointFuture.isDone()); final int checkpointedNumber = bytesToInt(checkpointFuture.get()); - assertEquals(checkpointedNumber, sender.events.size()); + assertEquals(checkpointedNumber, sender.getNumberOfSentEvents()); for (int i = 0; i < checkpointedNumber; i++) { - assertEquals(i, ((TestOperatorEvent) sender.events.get(i).event).getValue()); + assertEquals( + i, ((TestOperatorEvent) sender.getAllSentEvents().get(i).event).getValue()); } } @@ -413,22 +382,18 @@ private static TestingOperatorCoordinator getCoordinator(OperatorCoordinatorHold } private OperatorCoordinatorHolder createCoordinatorHolder( - final BiFunction< - SerializedValue, Integer, CompletableFuture> - eventSender, + final SubtaskAccess.SubtaskAccessFactory eventTarget, final Function coordinatorCtor) throws Exception { return createCoordinatorHolder( - eventSender, + eventTarget, coordinatorCtor, ComponentMainThreadExecutorServiceAdapter.forMainThread()); } private OperatorCoordinatorHolder createCoordinatorHolder( - final BiFunction< - SerializedValue, Integer, CompletableFuture> - eventSender, + final SubtaskAccess.SubtaskAccessFactory eventTarget, final Function coordinatorCtor, final ComponentMainThreadExecutor mainThreadExecutor) throws Exception { @@ -451,11 +416,11 @@ public OperatorCoordinator create(OperatorCoordinator.Context context) { OperatorCoordinatorHolder.create( opId, provider, - eventSender, "test-coordinator-name", getClass().getClassLoader(), 3, - 1775); + 1775, + eventTarget); holder.lazyInitialize(globalFailureHandler, mainThreadExecutor); holder.start(); @@ -511,7 +476,7 @@ protected void step() throws Exception { checkpoint.complete(intToBytes(num)); checkpoint = null; } - context.sendEvent(new TestOperatorEvent(num++), 0); + subtaskGateways[0].sendEvent(new TestOperatorEvent(num++)); condition.signalAll(); } finally { lock.unlock(); @@ -542,9 +507,9 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r protected void step() throws Exception { Thread.sleep(2); - context.sendEvent(new TestOperatorEvent(num++), 0); - context.sendEvent(new TestOperatorEvent(num++), 1); - context.sendEvent(new TestOperatorEvent(num++), 2); + subtaskGateways[0].sendEvent(new TestOperatorEvent(num++)); + subtaskGateways[1].sendEvent(new TestOperatorEvent(num++)); + subtaskGateways[2].sendEvent(new TestOperatorEvent(num++)); if (checkpoint != null) { checkpoint.complete(intToBytes(num)); @@ -559,18 +524,18 @@ private abstract static class CheckpointEventOrderTestBaseCoordinator private final Thread coordinatorThread; protected final Context context; + protected final SubtaskGateway[] subtaskGateways; private volatile boolean closed; CheckpointEventOrderTestBaseCoordinator(Context context) { this.context = context; + this.subtaskGateways = new SubtaskGateway[context.currentParallelism()]; this.coordinatorThread = new Thread(this); } @Override - public void start() throws Exception { - coordinatorThread.start(); - } + public void start() throws Exception {} @Override public void close() throws Exception { @@ -588,6 +553,20 @@ public void subtaskFailed(int subtask, @Nullable Throwable reason) {} @Override public void subtaskReset(int subtask, long checkpointId) {} + @Override + public void subtaskReady(int subtask, SubtaskGateway gateway) { + subtaskGateways[subtask] = gateway; + + for (SubtaskGateway subtaskGateway : subtaskGateways) { + if (subtaskGateway == null) { + return; + } + } + + // start only once all tasks are ready + coordinatorThread.start(); + } + @Override public abstract void checkpointCoordinator( long checkpointId, CompletableFuture result) throws Exception; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java index 8a1d3310fa720..456f30dc7e191 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java @@ -224,14 +224,21 @@ public void taskRepeatedFailureNotifyCoordinator() throws Exception { } @Test - public void taskExceptionWhenTasksNotRunning() throws Exception { + public void taskGatewayNotSetBeforeTasksRunning() throws Exception { final DefaultScheduler scheduler = createAndStartScheduler(); - final OperatorCoordinator.Context context = getCoordinator(scheduler).getContext(); + final TestingOperatorCoordinator coordinator = getCoordinator(scheduler); + final OperatorCoordinator.SubtaskGateway gateway = coordinator.getSubtaskGateway(0); - final CompletableFuture result = context.sendEvent(new TestOperatorEvent(), 0); - executor.triggerAll(); // process event sending + assertNull(gateway); + } - assertThat(result, futureFailedWith(TaskNotRunningException.class)); + @Test + public void taskGatewayAvailableWhenTasksRunning() throws Exception { + final DefaultScheduler scheduler = createSchedulerAndDeployTasks(); + final TestingOperatorCoordinator coordinator = getCoordinator(scheduler); + final OperatorCoordinator.SubtaskGateway gateway = coordinator.getSubtaskGateway(0); + + assertNotNull(gateway); } @Test @@ -239,8 +246,10 @@ public void taskTaskManagerFailuresAreReportedBack() throws Exception { final DefaultScheduler scheduler = createSchedulerAndDeployTasks(new FailingTaskExecutorOperatorEventGateway()); - final OperatorCoordinator.Context context = getCoordinator(scheduler).getContext(); - final CompletableFuture result = context.sendEvent(new TestOperatorEvent(), 0); + final TestingOperatorCoordinator coordinator = getCoordinator(scheduler); + final OperatorCoordinator.SubtaskGateway gateway = coordinator.getSubtaskGateway(0); + + final CompletableFuture result = gateway.sendEvent(new TestOperatorEvent()); executor.triggerAll(); // process event sending assertThat(result, futureFailedWith(TestException.class)); @@ -704,6 +713,10 @@ private void scheduleAllTasksToRunning(DefaultScheduler scheduler) { assertEquals( ExecutionState.RUNNING, SchedulerTestingUtils.getExecutionState(scheduler, testVertexId, 0)); + + // trigger actions depending on the switch to running, like the notifications + // that the task is reads and the task gateway setup + executor.triggerAll(); } private TestingOperatorCoordinator getCoordinator(DefaultScheduler scheduler) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java index 53ee685565e72..bc5f2fe0b0e6e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java @@ -19,16 +19,14 @@ package org.apache.flink.runtime.operators.coordination; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.operators.coordination.TestEventSender.EventWithSubtask; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks.EventWithSubtask; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.SerializedValue; import org.junit.Test; import java.util.concurrent.CompletableFuture; import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -37,54 +35,52 @@ public class OperatorEventValveTest { @Test - public void eventsPassThroughOpenValve() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); + public void eventsPassThroughOpenValve() { + final EventReceivingTasks sender = EventReceivingTasks.createForRunningTasks(); + final OperatorEventValve valve = new OperatorEventValve(); final OperatorEvent event = new TestOperatorEvent(); final CompletableFuture future = new CompletableFuture<>(); - valve.sendEvent(new SerializedValue<>(event), 11, future); + valve.sendEvent(sender.createSendAction(event, 11), future); assertThat(sender.events, contains(new EventWithSubtask(event, 11))); assertTrue(future.isDone()); } @Test(expected = IllegalStateException.class) - public void errorShuttingUnmarkedValve() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); + public void errorShuttingUnmarkedValve() { + final OperatorEventValve valve = new OperatorEventValve(); valve.shutValve(123L); } @Test(expected = IllegalStateException.class) - public void errorShuttingValveForOtherMark() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); + public void errorShuttingValveForOtherMark() { + final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(100L); valve.shutValve(123L); } @Test - public void eventsBlockedByClosedValve() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); + public void eventsBlockedByClosedValve() { + final EventReceivingTasks sender = EventReceivingTasks.createForRunningTasks(); + final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(1L); valve.shutValve(1L); final CompletableFuture future = new CompletableFuture<>(); - valve.sendEvent(new SerializedValue<>(new TestOperatorEvent()), 1, future); + valve.sendEvent(sender.createSendAction(new TestOperatorEvent(), 1), future); assertTrue(sender.events.isEmpty()); assertFalse(future.isDone()); } @Test - public void eventsReleasedAfterOpeningValve() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); + public void eventsReleasedAfterOpeningValve() { + final EventReceivingTasks sender = EventReceivingTasks.createForRunningTasks(); + final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(17L); valve.shutValve(17L); @@ -92,84 +88,32 @@ public void eventsReleasedAfterOpeningValve() throws Exception { final OperatorEvent event1 = new TestOperatorEvent(); final OperatorEvent event2 = new TestOperatorEvent(); final CompletableFuture future1 = new CompletableFuture<>(); - valve.sendEvent(new SerializedValue<>(event1), 3, future1); + valve.sendEvent(sender.createSendAction(event1, 3), future1); final CompletableFuture future2 = new CompletableFuture<>(); - valve.sendEvent(new SerializedValue<>(event2), 0, future2); + valve.sendEvent(sender.createSendAction(event2, 0), future2); valve.openValveAndUnmarkCheckpoint(); assertThat( sender.events, - containsInAnyOrder( - new EventWithSubtask(event1, 3), new EventWithSubtask(event2, 0))); + contains(new EventWithSubtask(event1, 3), new EventWithSubtask(event2, 0))); assertTrue(future1.isDone()); assertTrue(future2.isDone()); } @Test - public void releasedEventsForwardSendFailures() throws Exception { - final TestEventSender sender = new TestEventSender(new FlinkException("test")); - final OperatorEventValve valve = new OperatorEventValve(sender); + public void releasedEventsForwardSendFailures() { + final EventReceivingTasks sender = + EventReceivingTasks.createForRunningTasksFailingRpcs(new FlinkException("test")); + final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(17L); valve.shutValve(17L); final CompletableFuture future = new CompletableFuture<>(); - valve.sendEvent(new SerializedValue<>(new TestOperatorEvent()), 10, future); + valve.sendEvent(sender.createSendAction(new TestOperatorEvent(), 10), future); valve.openValveAndUnmarkCheckpoint(); assertTrue(future.isCompletedExceptionally()); } - - @Test - public void resetDropsAllEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); - valve.markForCheckpoint(17L); - valve.shutValve(17L); - - valve.sendEvent( - new SerializedValue<>(new TestOperatorEvent()), 0, new CompletableFuture<>()); - valve.sendEvent( - new SerializedValue<>(new TestOperatorEvent()), 1, new CompletableFuture<>()); - - valve.reset(); - valve.openValveAndUnmarkCheckpoint(); - - assertTrue(sender.events.isEmpty()); - } - - @Test - public void resetForTaskDropsSelectiveEvents() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); - valve.markForCheckpoint(17L); - valve.shutValve(17L); - - final OperatorEvent event1 = new TestOperatorEvent(); - final OperatorEvent event2 = new TestOperatorEvent(); - final CompletableFuture future1 = new CompletableFuture<>(); - valve.sendEvent(new SerializedValue<>(event1), 0, future1); - final CompletableFuture future2 = new CompletableFuture<>(); - valve.sendEvent(new SerializedValue<>(event2), 1, future2); - - valve.resetForTask(1); - valve.openValveAndUnmarkCheckpoint(); - - assertThat(sender.events, contains(new EventWithSubtask(event1, 0))); - assertTrue(future1.isDone()); - assertTrue(future2.isCompletedExceptionally()); - } - - @Test - public void resetOpensValve() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); - - valve.markForCheckpoint(17L); - valve.shutValve(17L); - valve.reset(); - - assertFalse(valve.isShut()); - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinatorTest.java index 5b8079c75d387..2fb8ea696c408 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinatorTest.java @@ -32,45 +32,52 @@ Licensed to the Apache Software Foundation (ASF) under one import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; /** Unit tests for {@link RecreateOnResetOperatorCoordinator}. */ public class RecreateOnResetOperatorCoordinatorTest { + private static final OperatorID OPERATOR_ID = new OperatorID(1234L, 5678L); private static final int NUM_SUBTASKS = 1; @Test - public void testQuiesceableContextNotQuiesced() throws TaskNotRunningException { + public void testQuiesceableContextForwardsProperties() { MockOperatorCoordinatorContext context = new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS); RecreateOnResetOperatorCoordinator.QuiesceableContext quiesceableContext = new RecreateOnResetOperatorCoordinator.QuiesceableContext(context); - TestingEvent event = new TestingEvent(); - quiesceableContext.sendEvent(event, 0); - quiesceableContext.failJob(new Exception()); - assertEquals(OPERATOR_ID, quiesceableContext.getOperatorId()); assertEquals(NUM_SUBTASKS, quiesceableContext.currentParallelism()); - assertEquals(Collections.singletonList(event), context.getEventsToOperatorBySubtaskId(0)); + } + + @Test + public void testQuiesceableContextNotQuiesced() { + MockOperatorCoordinatorContext context = + new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS); + RecreateOnResetOperatorCoordinator.QuiesceableContext quiesceableContext = + new RecreateOnResetOperatorCoordinator.QuiesceableContext(context); + + final Exception exception = new Exception(); + quiesceableContext.failJob(exception); + assertTrue(context.isJobFailed()); + assertSame(exception, context.getJobFailureReason()); } @Test - public void testQuiescedContext() throws TaskNotRunningException { + public void testQuiescedContext() { MockOperatorCoordinatorContext context = new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS); RecreateOnResetOperatorCoordinator.QuiesceableContext quiesceableContext = new RecreateOnResetOperatorCoordinator.QuiesceableContext(context); quiesceableContext.quiesce(); - quiesceableContext.sendEvent(new TestingEvent(), 0); quiesceableContext.failJob(new Exception()); - assertEquals(OPERATOR_ID, quiesceableContext.getOperatorId()); - assertEquals(NUM_SUBTASKS, quiesceableContext.currentParallelism()); - assertTrue(context.getEventsToOperator().isEmpty()); assertFalse(context.isJobFailed()); } @@ -99,6 +106,7 @@ public void testResetToCheckpoint() throws Exception { getInternalCoordinator(coordinator); assertEquals( stateToRestore, internalCoordinatorAfterReset.getLastRestoredCheckpointState()); + assertNotSame(internalCoordinatorAfterReset, internalCoordinatorBeforeReset); } @Test @@ -124,8 +132,7 @@ public void testMethodCallsOnLongResetToCheckpoint() throws Exception { final CountDownLatch blockOnCloseLatch = new CountDownLatch(1); // Let the user coordinator block on close. TestingCoordinatorProvider provider = new TestingCoordinatorProvider(blockOnCloseLatch); - MockOperatorCoordinatorContext context = - new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS); + MockOperatorCoordinatorContext context = new MockOperatorCoordinatorContext(OPERATOR_ID, 2); RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) provider.create(context, closingTimeoutMs); @@ -245,8 +252,6 @@ public void testConsecutiveResetToCheckpoint() throws Exception { "Timed out when waiting for the coordinator to close."); } - public void testFailureInCreateCoordinator() {} - // --------------- private RecreateOnResetOperatorCoordinator createCoordinator( @@ -262,9 +267,10 @@ private TestingOperatorCoordinator getInternalCoordinator( // --------------- + @SuppressWarnings("serial") private static class TestingCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { - private static final long serialVersionUID = 4184184580789587013L; + private final CountDownLatch blockOnCloseLatch; private final List createdCoordinators; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestEventSender.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestEventSender.java deleted file mode 100644 index 0feabed4e4d5e..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestEventSender.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.messages.Acknowledge; -import org.apache.flink.util.SerializedValue; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; -import java.util.function.BiFunction; - -/** - * A test implementation of the BiFunction interface used as the underlying event sender in the - * {@link OperatorCoordinatorHolder}. - */ -final class TestEventSender - implements BiFunction< - SerializedValue, Integer, CompletableFuture> { - - final ArrayList events = new ArrayList<>(); - - @Nullable private final Throwable failureCause; - - /** Creates a sender that collects events and acknowledges all events successfully. */ - TestEventSender() { - this(null); - } - - /** - * Creates a sender that collects events and fails all the send-futures with the given - * exception, if it is non-null. - */ - TestEventSender(@Nullable Throwable failureCause) { - this.failureCause = failureCause; - } - - @Override - public CompletableFuture apply( - SerializedValue event, Integer subtask) { - final OperatorEvent deserializedEvent; - try { - deserializedEvent = event.deserializeValue(getClass().getClassLoader()); - } catch (IOException | ClassNotFoundException e) { - throw new AssertionError(e); - } - events.add(new EventWithSubtask(deserializedEvent, subtask)); - - return failureCause == null - ? CompletableFuture.completedFuture(Acknowledge.get()) - : FutureUtils.completedExceptionally(failureCause); - } - - // ------------------------------------------------------------------------ - - /** A combination of an {@link OperatorEvent} and the target subtask it is sent to. */ - static final class EventWithSubtask { - - final OperatorEvent event; - final int subtask; - - EventWithSubtask(OperatorEvent event, int subtask) { - this.event = event; - this.subtask = subtask; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final EventWithSubtask that = (EventWithSubtask) o; - return subtask == that.subtask && event.equals(that.event); - } - - @Override - public int hashCode() { - return Objects.hash(event, subtask); - } - - @Override - public String toString() { - return event + " => subtask " + subtask; - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestingOperatorCoordinator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestingOperatorCoordinator.java index b2b81c06c9e70..50b3046fc4bb4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestingOperatorCoordinator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/TestingOperatorCoordinator.java @@ -24,7 +24,9 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -45,11 +47,13 @@ class TestingOperatorCoordinator implements OperatorCoordinator { @Nullable private byte[] lastRestoredCheckpointState; private long lastRestoredCheckpointId; - private BlockingQueue> triggeredCheckpoints; + private final BlockingQueue> triggeredCheckpoints; - private BlockingQueue lastCheckpointComplete; + private final BlockingQueue lastCheckpointComplete; - private BlockingQueue receivedOperatorEvents; + private final BlockingQueue receivedOperatorEvents; + + private final Map subtaskGateways; private boolean started; private boolean closed; @@ -65,6 +69,7 @@ public TestingOperatorCoordinator( this.lastCheckpointComplete = new LinkedBlockingQueue<>(); this.receivedOperatorEvents = new LinkedBlockingQueue<>(); this.blockOnCloseLatch = blockOnCloseLatch; + this.subtaskGateways = new HashMap<>(); } // ------------------------------------------------------------------------ @@ -90,6 +95,7 @@ public void handleEventFromOperator(int subtask, OperatorEvent event) { @Override public void subtaskFailed(int subtask, @Nullable Throwable reason) { failedTasks.add(subtask); + subtaskGateways.remove(subtask); } @Override @@ -97,6 +103,11 @@ public void subtaskReset(int subtask, long checkpointId) { restoredTasks.add(new SubtaskAndCheckpoint(subtask, checkpointId)); } + @Override + public void subtaskReady(int subtask, SubtaskGateway gateway) { + subtaskGateways.put(subtask, gateway); + } + @Override public void checkpointCoordinator(long checkpointId, CompletableFuture result) { boolean added = triggeredCheckpoints.offer(result); @@ -120,6 +131,10 @@ public OperatorCoordinator.Context getContext() { return context; } + public SubtaskGateway getSubtaskGateway(int subtask) { + return subtaskGateways.get(subtask); + } + public boolean isStarted() { return started; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java new file mode 100644 index 0000000000000..e9942888203ba --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java @@ -0,0 +1,105 @@ +/* + * 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.util; + +import org.junit.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** Unit tests for the {@link IncompleteFuturesTracker}. */ +public class IncompleteFuturesTrackerTest { + + @Test + public void testFutureTracked() { + final IncompleteFuturesTracker tracker = new IncompleteFuturesTracker(); + final CompletableFuture future = new CompletableFuture<>(); + + tracker.trackFutureWhileIncomplete(future); + + assertThat(tracker.getTrackedFutures(), contains(future)); + } + + @Test + public void testFutureRemovedAfterCompletion() { + final IncompleteFuturesTracker tracker = new IncompleteFuturesTracker(); + final CompletableFuture future = new CompletableFuture<>(); + + tracker.trackFutureWhileIncomplete(future); + future.complete(null); + + assertThat(tracker.getTrackedFutures(), not(contains(future))); + } + + @Test + public void testFutureNotAddedIfAlreadyCompleted() { + final IncompleteFuturesTracker tracker = new IncompleteFuturesTracker(); + final CompletableFuture future = new CompletableFuture<>(); + + future.complete(null); + tracker.trackFutureWhileIncomplete(future); + + assertThat(tracker.getTrackedFutures(), not(contains(future))); + } + + @Test + public void testFailFutures() throws Exception { + final IncompleteFuturesTracker tracker = new IncompleteFuturesTracker(); + final CompletableFuture future = new CompletableFuture<>(); + + tracker.trackFutureWhileIncomplete(future); + + final Exception expectedException = new Exception(); + tracker.failAllFutures(expectedException); + + assertTrue(future.isCompletedExceptionally()); + try { + future.get(); + fail(); + } catch (ExecutionException e) { + assertSame(expectedException, e.getCause()); + } + } + + @Test + public void testFailFuturesImmediately() throws Exception { + final IncompleteFuturesTracker tracker = new IncompleteFuturesTracker(); + final CompletableFuture future = new CompletableFuture<>(); + + final Exception expectedException = new Exception(); + tracker.failAllFutures(expectedException); + + tracker.trackFutureWhileIncomplete(future); + + assertTrue(future.isCompletedExceptionally()); + try { + future.get(); + fail(); + } catch (ExecutionException e) { + assertSame(expectedException, e.getCause()); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java index 2297514d6ccc7..382d2f957bdd7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java @@ -1,20 +1,20 @@ /* -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. -*/ + * 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.source.coordinator; @@ -25,10 +25,13 @@ Licensed to the Apache Software Foundation (ASF) under one import org.apache.flink.core.testutils.ManuallyTriggeredScheduledExecutorService; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.source.event.AddSplitEvent; +import org.apache.flink.runtime.source.event.ReaderRegistrationEvent; +import org.hamcrest.Matchers; import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicReference; @@ -38,27 +41,34 @@ Licensed to the Apache Software Foundation (ASF) under one import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** Unit test for {@link SourceCoordinatorContext}. */ public class SourceCoordinatorContextTest extends SourceCoordinatorTestBase { @Test - public void testRegisterReader() { + public void testRegisterReader() throws Exception { + sourceReady(); List readerInfo = registerReaders(); assertTrue(context.registeredReaders().containsKey(0)); assertTrue(context.registeredReaders().containsKey(1)); assertEquals(readerInfo.get(0), context.registeredReaders().get(0)); assertEquals(readerInfo.get(1), context.registeredReaders().get(1)); + + final TestingSplitEnumerator enumerator = getEnumerator(); + assertThat(enumerator.getRegisteredReaders(), Matchers.containsInAnyOrder(0, 1, 2)); } @Test - public void testUnregisterReader() { + public void testTaskFailureUnregistersReader() throws Exception { + sourceReady(); List readerInfo = registerReaders(); - assertEquals(readerInfo.get(0), context.registeredReaders().get(0)); - context.unregisterSourceReader(0); + sourceCoordinator.subtaskFailed(0, null); + waitForCoordinatorToProcessActions(); + assertEquals("Only reader 2 should be registered.", 2, context.registeredReaders().size()); assertNull(context.registeredReaders().get(0)); assertEquals(readerInfo.get(1), context.registeredReaders().get(1)); @@ -80,8 +90,9 @@ public void testAssignSplitsFromOtherThread() throws Exception { testAssignSplits(false); } + @SuppressWarnings("unchecked") private void testAssignSplits(boolean fromCoordinatorExecutor) throws Exception { - // Register the readers. + sourceReady(); registerReaders(); // Assign splits to the readers. @@ -94,7 +105,8 @@ private void testAssignSplits(boolean fromCoordinatorExecutor) throws Exception // The tracker should have recorded the assignments. verifyAssignment( - Arrays.asList("0"), splitSplitAssignmentTracker.uncheckpointedAssignments().get(0)); + Collections.singletonList("0"), + splitSplitAssignmentTracker.uncheckpointedAssignments().get(0)); verifyAssignment( Arrays.asList("1", "2"), splitSplitAssignmentTracker.uncheckpointedAssignments().get(1)); @@ -102,31 +114,32 @@ private void testAssignSplits(boolean fromCoordinatorExecutor) throws Exception assertEquals( "There should be two events sent to the subtasks.", 2, - operatorCoordinatorContext.getEventsToOperator().size()); + receivingTasks.getNumberOfSentEvents()); // Assert the events to subtask0. - List eventsToSubtask0 = - operatorCoordinatorContext.getEventsToOperatorBySubtaskId(0); + List eventsToSubtask0 = receivingTasks.getSentEventsForSubtask(0); assertEquals(1, eventsToSubtask0.size()); OperatorEvent event = eventsToSubtask0.get(0); assertTrue(event instanceof AddSplitEvent); verifyAssignment( - Arrays.asList("0"), - ((AddSplitEvent) event).splits(new MockSourceSplitSerializer())); + Collections.singletonList("0"), + ((AddSplitEvent) event).splits(new MockSourceSplitSerializer())); } @Test - public void testAssignSplitToUnregisteredReaderFromCoordinatorExecutor() { + public void testAssignSplitToUnregisteredReaderFromCoordinatorExecutor() throws Exception { testAssignSplitToUnregisterdReader(true); } @Test - public void testAssignSplitToUnregisteredReaderFromOtherThread() { + public void testAssignSplitToUnregisteredReaderFromOtherThread() throws Exception { testAssignSplitToUnregisterdReader(false); } - private void testAssignSplitToUnregisterdReader(boolean fromCoordinatorExecutor) { - // Assign splits to the readers. + private void testAssignSplitToUnregisterdReader(boolean fromCoordinatorExecutor) + throws Exception { + sourceReady(); + SplitsAssignment splitsAssignment = getSplitsAssignment(2, 0); verifyException( () -> { @@ -183,13 +196,19 @@ public void testCallableInterruptedDuringShutdownDoNotFailJob() throws Interrupt // ------------------------ private List registerReaders() { - // Register the readers. - ReaderInfo readerInfo0 = new ReaderInfo(0, "subtask_0_location"); - ReaderInfo readerInfo1 = new ReaderInfo(1, "subtask_1_location"); - ReaderInfo readerInfo2 = new ReaderInfo(2, "subtask_1_location"); - context.registerSourceReader(readerInfo0); - context.registerSourceReader(readerInfo1); - context.registerSourceReader(readerInfo2); - return Arrays.asList(readerInfo0, readerInfo1, readerInfo2); + final List infos = + Arrays.asList( + new ReaderInfo(0, "subtask_0_location"), + new ReaderInfo(1, "subtask_1_location"), + new ReaderInfo(2, "subtask_2_location")); + + for (ReaderInfo info : infos) { + sourceCoordinator.handleEventFromOperator( + info.getSubtaskId(), + new ReaderRegistrationEvent(info.getSubtaskId(), info.getLocation())); + } + waitForCoordinatorToProcessActions(); + + return infos; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTest.java index 66557073cb73a..20d65f8ac1d1b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTest.java @@ -1,20 +1,20 @@ /* -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. -*/ + * 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.source.coordinator; @@ -35,21 +35,18 @@ Licensed to the Apache Software Foundation (ASF) under one import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.runtime.source.event.AddSplitEvent; -import org.apache.flink.runtime.source.event.ReaderRegistrationEvent; import org.apache.flink.runtime.source.event.SourceEventWrapper; import org.junit.Test; import javax.annotation.Nullable; -import java.io.IOException; import java.net.URL; import java.net.URLClassLoader; import java.time.Duration; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -101,55 +98,41 @@ public void testRestCheckpointAfterCoordinatorStarted() throws Exception { "The coordinator can only be reset if it was not yet started"); } - @Test(timeout = 10000L) + @Test public void testStart() throws Exception { sourceCoordinator.start(); - while (!getEnumerator().started()) { - Thread.sleep(1); - } + waitForCoordinatorToProcessActions(); + + assertTrue(getEnumerator().isStarted()); } @Test public void testClosed() throws Exception { sourceCoordinator.start(); sourceCoordinator.close(); - assertTrue(getEnumerator().closed()); - } - - @Test - public void testReaderRegistration() throws Exception { - sourceCoordinator.start(); - sourceCoordinator.handleEventFromOperator(0, new ReaderRegistrationEvent(0, "location_0")); - check( - () -> { - assertEquals( - "2 splits should have been assigned to reader 0", - 4, - getEnumerator().getUnassignedSplits().size()); - assertTrue(context.registeredReaders().containsKey(0)); - assertTrue(getEnumerator().getHandledSourceEvent().isEmpty()); - verifyAssignment( - Arrays.asList("0", "3"), - splitSplitAssignmentTracker.uncheckpointedAssignments().get(0)); - }); + assertTrue(getEnumerator().isClosed()); } @Test public void testHandleSourceEvent() throws Exception { - sourceCoordinator.start(); + sourceReady(); + SourceEvent sourceEvent = new SourceEvent() {}; sourceCoordinator.handleEventFromOperator(0, new SourceEventWrapper(sourceEvent)); - check( - () -> { - assertEquals(1, getEnumerator().getHandledSourceEvent().size()); - assertEquals(sourceEvent, getEnumerator().getHandledSourceEvent().get(0)); - }); + waitForCoordinatorToProcessActions(); + + assertEquals(1, getEnumerator().getHandledSourceEvent().size()); + assertEquals(sourceEvent, getEnumerator().getHandledSourceEvent().get(0)); } @Test public void testCheckpointCoordinatorAndRestore() throws Exception { - sourceCoordinator.start(); - sourceCoordinator.handleEventFromOperator(0, new ReaderRegistrationEvent(0, "location_0")); + sourceReady(); + addTestingSplitSet(6); + + registerReader(0); + getEnumerator().executeAssignOneSplit(0); + getEnumerator().executeAssignOneSplit(0); final CompletableFuture checkpointFuture = new CompletableFuture<>(); sourceCoordinator.checkpointCoordinator(100L, checkpointFuture); @@ -158,14 +141,14 @@ public void testCheckpointCoordinatorAndRestore() throws Exception { // restore from the checkpoints. SourceCoordinator restoredCoordinator = getNewSourceCoordinator(); restoredCoordinator.resetToCheckpoint(100L, bytes); - MockSplitEnumerator restoredEnumerator = - (MockSplitEnumerator) restoredCoordinator.getEnumerator(); + TestingSplitEnumerator restoredEnumerator = + (TestingSplitEnumerator) restoredCoordinator.getEnumerator(); SourceCoordinatorContext restoredContext = restoredCoordinator.getContext(); assertEquals( "2 splits should have been assigned to reader 0", 4, restoredEnumerator.getUnassignedSplits().size()); - assertTrue(restoredEnumerator.getHandledSourceEvent().isEmpty()); + assertTrue(restoredEnumerator.getContext().registeredReaders().isEmpty()); assertEquals( "Registered readers should not be recovered by restoring", 0, @@ -173,119 +156,75 @@ public void testCheckpointCoordinatorAndRestore() throws Exception { } @Test - @SuppressWarnings("unchecked") public void testSubtaskFailedAndRevertUncompletedAssignments() throws Exception { - sourceCoordinator.start(); - - // Assign some splits to reader 0 then take snapshot 100. - sourceCoordinator.handleEventFromOperator(0, new ReaderRegistrationEvent(0, "location_0")); - - final CompletableFuture checkpointFuture1 = new CompletableFuture<>(); - sourceCoordinator.checkpointCoordinator(100L, checkpointFuture1); - checkpointFuture1.get(); + sourceReady(); + addTestingSplitSet(6); - // Add split 6, assign it to reader 0 and take another snapshot 101. - getEnumerator().addNewSplits(Collections.singletonList(new MockSourceSplit(6))); + // two splits pending for checkpoint 100 + registerReader(0); + getEnumerator().executeAssignOneSplit(0); + getEnumerator().executeAssignOneSplit(0); + sourceCoordinator.checkpointCoordinator(100L, new CompletableFuture<>()); - final CompletableFuture checkpointFuture2 = new CompletableFuture<>(); - sourceCoordinator.checkpointCoordinator(101L, checkpointFuture2); - checkpointFuture2.get(); + getEnumerator().addNewSplits(new MockSourceSplit(6)); + getEnumerator().executeAssignOneSplit(0); + sourceCoordinator.checkpointCoordinator(101L, new CompletableFuture<>()); // check the state. - check( - () -> { - // There should be 4 unassigned splits. - assertEquals(4, getEnumerator().getUnassignedSplits().size()); - verifyAssignment( - Arrays.asList("0", "3"), - splitSplitAssignmentTracker - .assignmentsByCheckpointId() - .get(100L) - .get(0)); - assertTrue(splitSplitAssignmentTracker.uncheckpointedAssignments().isEmpty()); - verifyAssignment( - Arrays.asList("0", "3"), - splitSplitAssignmentTracker.assignmentsByCheckpointId(100L).get(0)); - verifyAssignment( - Arrays.asList("6"), - splitSplitAssignmentTracker.assignmentsByCheckpointId(101L).get(0)); - - List eventsToReader0 = - operatorCoordinatorContext.getEventsToOperator().get(0); - assertEquals(2, eventsToReader0.size()); - try { - verifyAssignment( - Arrays.asList("0", "3"), - ((AddSplitEvent) eventsToReader0.get(0)) - .splits(new MockSourceSplitSerializer())); - verifyAssignment( - Arrays.asList("6"), - ((AddSplitEvent) eventsToReader0.get(1)) - .splits(new MockSourceSplitSerializer())); - } catch (IOException e) { - fail("Failed to deserialize splits."); - } - }); - - // Fail reader 0. + waitForCoordinatorToProcessActions(); + + assertEquals(4, getEnumerator().getUnassignedSplits().size()); + assertTrue(splitSplitAssignmentTracker.uncheckpointedAssignments().isEmpty()); + verifyAssignment( + Arrays.asList("0", "1"), + splitSplitAssignmentTracker.assignmentsByCheckpointId().get(100L).get(0)); + verifyAssignment( + Collections.singletonList("2"), + splitSplitAssignmentTracker.assignmentsByCheckpointId(101L).get(0)); + + // none of the checkpoints is confirmed, we fail and revert to the previous one sourceCoordinator.subtaskFailed(0, null); - sourceCoordinator.subtaskReset(0, 99L); // checkpoint ID before the triggered checkpoints - - // check the state again. - check( - () -> { - // - assertFalse( - "Reader 0 should have been unregistered.", - context.registeredReaders().containsKey(0)); - // The tracker should have reverted all the splits assignment to reader 0. - for (Map assignment : - splitSplitAssignmentTracker.assignmentsByCheckpointId().values()) { - assertFalse( - "Assignment in uncompleted checkpoint should have been reverted.", - assignment.containsKey(0)); - } - assertFalse( - splitSplitAssignmentTracker.uncheckpointedAssignments().containsKey(0)); - // The split enumerator should now contains the splits used to be assigned to - // reader 0. - assertEquals(7, getEnumerator().getUnassignedSplits().size()); - }); + sourceCoordinator.subtaskReset(0, 99L); + waitForCoordinatorToProcessActions(); + + assertFalse( + "Reader 0 should have been unregistered.", + context.registeredReaders().containsKey(0)); + // The tracker should have reverted all the splits assignment to reader 0. + for (Map assignment : + splitSplitAssignmentTracker.assignmentsByCheckpointId().values()) { + assertFalse( + "Assignment in uncompleted checkpoint should have been reverted.", + assignment.containsKey(0)); + } + assertFalse(splitSplitAssignmentTracker.uncheckpointedAssignments().containsKey(0)); + // The split enumerator should now contains the splits used to b + // assigned to reader 0. + assertEquals(7, getEnumerator().getUnassignedSplits().size()); } @Test public void testFailedSubtaskDoNotRevertCompletedCheckpoint() throws Exception { - sourceCoordinator.start(); + sourceReady(); + addTestingSplitSet(6); // Assign some splits to reader 0 then take snapshot 100. - sourceCoordinator.handleEventFromOperator(0, new ReaderRegistrationEvent(0, "location_0")); + registerReader(0); + getEnumerator().executeAssignOneSplit(0); + getEnumerator().executeAssignOneSplit(0); - final CompletableFuture checkpointFuture = new CompletableFuture<>(); - sourceCoordinator.checkpointCoordinator(100L, checkpointFuture); - checkpointFuture.get(); - - // Complete checkpoint 100. + sourceCoordinator.checkpointCoordinator(100L, new CompletableFuture<>()); sourceCoordinator.notifyCheckpointComplete(100L); - waitUtil( - () -> !getEnumerator().getSuccessfulCheckpoints().isEmpty(), - Duration.ofMillis(1000L), - "The enumerator failed to process the successful checkpoint " - + "before times out."); - assertEquals(100L, (long) getEnumerator().getSuccessfulCheckpoints().get(0)); - // Fail reader 0. sourceCoordinator.subtaskFailed(0, null); - check( - () -> { - // Reader 0 hase been unregistered. - assertFalse(context.registeredReaders().containsKey(0)); - // The assigned splits are not reverted. - assertEquals(4, getEnumerator().getUnassignedSplits().size()); - assertFalse( - splitSplitAssignmentTracker.uncheckpointedAssignments().containsKey(0)); - assertTrue(splitSplitAssignmentTracker.assignmentsByCheckpointId().isEmpty()); - }); + waitForCoordinatorToProcessActions(); + + assertEquals(100L, (long) getEnumerator().getSuccessfulCheckpoints().get(0)); + assertFalse(context.registeredReaders().containsKey(0)); + assertEquals(4, getEnumerator().getUnassignedSplits().size()); + assertFalse(splitSplitAssignmentTracker.uncheckpointedAssignments().containsKey(0)); + assertTrue(splitSplitAssignmentTracker.assignmentsByCheckpointId().isEmpty()); } @Test @@ -391,68 +330,44 @@ public void testUserClassLoaderWhenRestoringEnumerator() throws Exception { @Test public void testSerdeBackwardCompatibility() throws Exception { - // Preparation - sourceCoordinator.start(); - sourceCoordinator.handleEventFromOperator(0, new ReaderRegistrationEvent(0, "location_0")); - - // Make sure the reader has been registered and the split has been assigned - check( - () -> { - assertTrue(sourceCoordinator.getContext().registeredReaders().containsKey(0)); - assertEquals( - "2 splits should have been assigned to reader 0", - 4, - getEnumerator().getUnassignedSplits().size()); - }); + sourceReady(); + addTestingSplitSet(6); // Build checkpoint data with serde version 0 - final byte[] checkpointDataForV0Serde = createCheckpointDataWithSerdeV0(sourceCoordinator); + final TestingSplitEnumerator enumerator = getEnumerator(); + final Set splits = new HashSet<>(); + enumerator.runInEnumThreadAndSync(() -> splits.addAll(enumerator.snapshotState())); + + final byte[] checkpointDataForV0Serde = createCheckpointDataWithSerdeV0(splits); // Restore from checkpoint data with serde version 0 to test backward compatibility SourceCoordinator restoredCoordinator = getNewSourceCoordinator(); restoredCoordinator.resetToCheckpoint(15213L, checkpointDataForV0Serde); - MockSplitEnumerator restoredEnumerator = - (MockSplitEnumerator) restoredCoordinator.getEnumerator(); + TestingSplitEnumerator restoredEnumerator = + (TestingSplitEnumerator) restoredCoordinator.getEnumerator(); SourceCoordinatorContext restoredContext = restoredCoordinator.getContext(); // Check if enumerator is restored correctly - assertEquals( - "2 splits should have been assigned to reader 0", - 4, - restoredEnumerator.getUnassignedSplits().size()); + assertEquals(splits, restoredEnumerator.getUnassignedSplits()); assertTrue(restoredEnumerator.getHandledSourceEvent().isEmpty()); - assertEquals( - "Registered readers should not be recovered by restoring", - 0, - restoredContext.registeredReaders().size()); + assertEquals(0, restoredContext.registeredReaders().size()); } // ------------------------------------------------------------------------ // test helpers // ------------------------------------------------------------------------ - @SuppressWarnings("unchecked") - private byte[] createCheckpointDataWithSerdeV0(SourceCoordinator sourceCoordinator) - throws Exception { - - final DataOutputSerializer serializer = new DataOutputSerializer(32); - - serializer.writeInt(SourceCoordinatorSerdeUtils.VERSION_0); + private byte[] createCheckpointDataWithSerdeV0(Set splits) throws Exception { final MockSplitEnumeratorCheckpointSerializer enumChkptSerializer = new MockSplitEnumeratorCheckpointSerializer(); + final DataOutputSerializer serializer = new DataOutputSerializer(32); + serializer.writeInt(SourceCoordinatorSerdeUtils.VERSION_0); serializer.writeInt(enumChkptSerializer.getVersion()); - final byte[] serializedEnumChkpt = - enumChkptSerializer.serialize( - ((SourceCoordinator>) - sourceCoordinator) - .getEnumerator() - .snapshotState()); - + final byte[] serializedEnumChkpt = enumChkptSerializer.serialize(splits); serializer.writeInt(serializedEnumChkpt.length); - serializer.write(serializedEnumChkpt); // Version 0 wrote number of reader, see FLINK-21452 diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTestBase.java index cb95a6c2e395f..f0db377b320ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorTestBase.java @@ -1,37 +1,42 @@ /* -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. -*/ + * 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.source.coordinator; -import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.connector.source.mocks.MockSource; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer; -import org.apache.flink.api.connector.source.mocks.MockSplitEnumerator; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorCheckpointSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.source.event.ReaderRegistrationEvent; import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.ExceptionUtils; import org.junit.After; import org.junit.Before; +import java.util.ArrayList; +import java.util.List; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -41,20 +46,30 @@ Licensed to the Apache Software Foundation (ASF) under one /** The test base for SourceCoordinator related tests. */ public abstract class SourceCoordinatorTestBase { + protected static final String OPERATOR_NAME = "TestOperator"; protected static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); protected static final int NUM_SUBTASKS = 3; + // ---- Mocks for the underlying Operator Coordinator Context --- + protected EventReceivingTasks receivingTasks; + protected MockOperatorCoordinatorContext operatorCoordinatorContext; + + // ---- Mocks for the Source Coordinator Context ---- protected SourceCoordinatorProvider.CoordinatorExecutorThreadFactory coordinatorThreadFactory; protected ExecutorService coordinatorExecutor; - protected MockOperatorCoordinatorContext operatorCoordinatorContext; protected SplitAssignmentTracker splitSplitAssignmentTracker; protected SourceCoordinatorContext context; + + // ---- Mocks for the Source Coordinator ---- protected SourceCoordinator> sourceCoordinator; - private MockSplitEnumerator enumerator; + private TestingSplitEnumerator enumerator; + + // ------------------------------------------------------------------------ @Before public void setup() throws Exception { + receivingTasks = EventReceivingTasks.createForRunningTasks(); operatorCoordinatorContext = new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS); splitSplitAssignmentTracker = new SplitAssignmentTracker<>(); @@ -76,20 +91,66 @@ public void cleanUp() throws InterruptedException, TimeoutException { } } - protected MockSplitEnumerator getEnumerator() { + // ------------------------------------------------------------------------ + + protected TestingSplitEnumerator getEnumerator() { if (enumerator == null) { - enumerator = (MockSplitEnumerator) sourceCoordinator.getEnumerator(); + enumerator = + (TestingSplitEnumerator) sourceCoordinator.getEnumerator(); assertNotNull("source was not started", enumerator); } return enumerator; } - // -------------------------- + protected void sourceReady() throws Exception { + sourceCoordinator.start(); + setAllReaderTasksReady(sourceCoordinator); + } + + protected void setAllReaderTasksReady() { + setAllReaderTasksReady(sourceCoordinator); + } + + protected void setAllReaderTasksReady(SourceCoordinator sourceCoordinator) { + for (int i = 0; i < NUM_SUBTASKS; i++) { + sourceCoordinator.subtaskReady(i, receivingTasks.createGatewayForSubtask(i)); + } + } + + protected void addTestingSplitSet(int num) { + final List splits = new ArrayList<>(); + for (int i = 0; i < num; i++) { + splits.add(new MockSourceSplit(i)); + } + + getEnumerator().addNewSplits(splits); + } + + protected void registerReader(int subtask) { + sourceCoordinator.handleEventFromOperator( + subtask, new ReaderRegistrationEvent(subtask, "location_" + subtask)); + } + + protected void waitForCoordinatorToProcessActions() { + final CompletableFuture future = new CompletableFuture<>(); + context.runInCoordinatorThread(() -> future.complete(null)); + + try { + future.get(); + } catch (InterruptedException e) { + throw new AssertionError("test interrupted"); + } catch (ExecutionException e) { + ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); + } + } + + // ------------------------------------------------------------------------ - protected SourceCoordinator> getNewSourceCoordinator() - throws Exception { - Source> mockSource = - new MockSource(Boundedness.BOUNDED, NUM_SUBTASKS * 2); + protected SourceCoordinator> getNewSourceCoordinator() { + final Source> mockSource = + TestingSplitEnumerator.factorySource( + new MockSourceSplitSerializer(), + new MockSplitEnumeratorCheckpointSerializer()); return new SourceCoordinator<>( OPERATOR_NAME, coordinatorExecutor, mockSource, getNewSourceCoordinatorContext()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/TestingSplitEnumerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/TestingSplitEnumerator.java new file mode 100644 index 0000000000000..eda29a9963f22 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/TestingSplitEnumerator.java @@ -0,0 +1,255 @@ +/* + * 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.source.coordinator; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** + * A mock {@link SplitEnumerator} for unit tests. + * + *

    In contrast to the {@link org.apache.flink.api.connector.source.mocks.MockSplitEnumerator} + * class which this largely copies, this class does not implement any source logic directly, like + * split assignments, etc. This class simply captures which modifications happened to support test + * assertions. + */ +public class TestingSplitEnumerator + implements SplitEnumerator> { + + private final SplitEnumeratorContext context; + + private final Queue splits; + private final List handledEvents; + private final List successfulCheckpoints; + private final Set registeredReaders; + + private volatile boolean started; + private volatile boolean closed; + + public TestingSplitEnumerator(SplitEnumeratorContext context) { + this(context, Collections.emptySet()); + } + + public TestingSplitEnumerator( + SplitEnumeratorContext context, Collection restoredSplits) { + this.context = context; + this.splits = new ArrayDeque<>(restoredSplits); + this.handledEvents = new ArrayList<>(); + this.successfulCheckpoints = new ArrayList<>(); + this.registeredReaders = new HashSet<>(); + } + + // ------------------------------------------------------------------------ + + @Override + public void start() { + this.started = true; + } + + @Override + public void close() throws IOException { + this.closed = true; + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {} + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + handledEvents.add(sourceEvent); + } + + @Override + public void addSplitsBack(List splitsToAddBack, int subtaskId) { + splits.addAll(splitsToAddBack); + } + + @Override + public void addReader(int subtaskId) { + registeredReaders.add(subtaskId); + } + + @Override + public Set snapshotState() { + return new HashSet<>(splits); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + successfulCheckpoints.add(checkpointId); + } + + // ------------------------------------------------------------------------ + + public SplitEnumeratorContext getContext() { + return context; + } + + public boolean isStarted() { + return started; + } + + public boolean isClosed() { + return closed; + } + + public Set getUnassignedSplits() { + return new HashSet<>(splits); + } + + public List getHandledSourceEvent() { + return Collections.unmodifiableList(handledEvents); + } + + public List getSuccessfulCheckpoints() { + return Collections.unmodifiableList(successfulCheckpoints); + } + + public Set getRegisteredReaders() { + return Collections.unmodifiableSet(registeredReaders); + } + + // ------------------------------------------------------------------------ + // simple test actions to trigger + // ------------------------------------------------------------------------ + + @SafeVarargs + public final void addNewSplits(SplitT... newSplits) { + addNewSplits(Arrays.asList(newSplits)); + } + + public void addNewSplits(Collection newSplits) { + runInEnumThreadAndSync(() -> splits.addAll(newSplits)); + } + + public void executeAssignOneSplit(int subtask) { + runInEnumThreadAndSync( + () -> { + Preconditions.checkState(!splits.isEmpty(), "no splits available"); + final SplitT split = splits.poll(); + context.assignSplit(split, subtask); + }); + } + + public void runInEnumThreadAndSync(Runnable action) { + final CompletableFuture future = new CompletableFuture<>(); + context.runInCoordinatorThread( + () -> { + try { + action.run(); + future.complete(null); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + + try { + future.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); + } + } + + // ------------------------------------------------------------------------ + // Source that solely acts as a factory for this enumerator + // + // This is needed for now because the SourceCoordinator does not accept + // slim dependencies (like only the serializer, or suppliers for the + // enumerator), but needs a reference to the source as a whole. + // ------------------------------------------------------------------------ + + public static Source> factorySource( + SimpleVersionedSerializer splitSerializer, + SimpleVersionedSerializer> checkpointSerializer) { + return new FactorySource<>(splitSerializer, checkpointSerializer); + } + + @SuppressWarnings("serial") + private static final class FactorySource + implements Source> { + + private final SimpleVersionedSerializer splitSerializer; + private final SimpleVersionedSerializer> checkpointSerializer; + + public FactorySource( + SimpleVersionedSerializer splitSerializer, + SimpleVersionedSerializer> checkpointSerializer) { + this.splitSerializer = splitSerializer; + this.checkpointSerializer = checkpointSerializer; + } + + @Override + public Boundedness getBoundedness() { + throw new UnsupportedOperationException(); + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) { + throw new UnsupportedOperationException(); + } + + @Override + public TestingSplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new TestingSplitEnumerator<>(enumContext); + } + + @Override + public SplitEnumerator> restoreEnumerator( + SplitEnumeratorContext enumContext, Set checkpoint) { + return new TestingSplitEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return splitSerializer; + } + + @Override + public SimpleVersionedSerializer> getEnumeratorCheckpointSerializer() { + return checkpointSerializer; + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java index 42058a57ff0a7..0c05c043bb43e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java @@ -193,6 +193,11 @@ public void subtaskReset(int subtask, long checkpointId) { // nothing to do here, connections are re-created lazily } + @Override + public void subtaskReady(int subtask, SubtaskGateway gateway) { + // nothing to do here, connections are re-created lazily + } + @Override public void checkpointCoordinator(long checkpointId, CompletableFuture result) throws Exception { From 1919af2bd73d29e630be34d9beb0d4ead4bf5042 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 13 Apr 2021 10:37:26 +0200 Subject: [PATCH 06/10] [hotfix][coordination] Make failed event valve shutting smoother. Failing to shut the event valve is quite common, it happens whenever a checkpoint gets canceled. So we don't handle this as an IllegalState, because it is very much an expected state. --- .../OperatorCoordinatorHolder.java | 14 ++++---- .../coordination/OperatorEventValve.java | 13 ++------ .../coordination/OperatorEventValveTest.java | 32 +++++++++++++------ 3 files changed, 34 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java index 89e1fc49c8b71..a6e2317b16df4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java @@ -282,13 +282,15 @@ private void checkpointCoordinatorInternal( () -> { if (failure != null) { result.completeExceptionally(failure); + } else if (eventValve.tryShutValve(checkpointId)) { + result.complete(success); } else { - try { - eventValve.shutValve(checkpointId); - result.complete(success); - } catch (Exception e) { - result.completeExceptionally(e); - } + // 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")); } })); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java index 98c90d5499617..8e71e36ea6a12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java @@ -128,21 +128,14 @@ public void markForCheckpoint(long checkpointId) { * Shuts the value. All events sent through this valve are blocked until the valve is re-opened. * If the valve is already shut, this does nothing. */ - public void shutValve(long checkpointId) { + public boolean tryShutValve(long checkpointId) { checkRunsInMainThread(); if (checkpointId == currentCheckpointId) { shut = true; - } else { - throw new IllegalStateException( - String.format( - "Cannot shut valve for non-prepared checkpoint. " - + "Prepared checkpoint = %s, attempting-to-close checkpoint = %d", - (currentCheckpointId == NO_CHECKPOINT - ? "(none)" - : String.valueOf(currentCheckpointId)), - checkpointId)); + return true; } + return false; } public void openValveAndUnmarkCheckpoint(long expectedCheckpointId) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java index bc5f2fe0b0e6e..dbe207dbbd6f9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventValveTest.java @@ -47,19 +47,33 @@ public void eventsPassThroughOpenValve() { assertTrue(future.isDone()); } - @Test(expected = IllegalStateException.class) - public void errorShuttingUnmarkedValve() { + @Test + public void shuttingMarkedValve() { + final OperatorEventValve valve = new OperatorEventValve(); + + valve.markForCheckpoint(200L); + final boolean shut = valve.tryShutValve(200L); + + assertTrue(shut); + } + + @Test + public void notShuttingUnmarkedValve() { final OperatorEventValve valve = new OperatorEventValve(); - valve.shutValve(123L); + final boolean shut = valve.tryShutValve(123L); + + assertFalse(shut); } - @Test(expected = IllegalStateException.class) - public void errorShuttingValveForOtherMark() { + @Test + public void notShuttingValveForOtherMark() { final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(100L); - valve.shutValve(123L); + final boolean shut = valve.tryShutValve(123L); + + assertFalse(shut); } @Test @@ -68,7 +82,7 @@ public void eventsBlockedByClosedValve() { final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(1L); - valve.shutValve(1L); + valve.tryShutValve(1L); final CompletableFuture future = new CompletableFuture<>(); valve.sendEvent(sender.createSendAction(new TestOperatorEvent(), 1), future); @@ -83,7 +97,7 @@ public void eventsReleasedAfterOpeningValve() { final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(17L); - valve.shutValve(17L); + valve.tryShutValve(17L); final OperatorEvent event1 = new TestOperatorEvent(); final OperatorEvent event2 = new TestOperatorEvent(); @@ -108,7 +122,7 @@ public void releasedEventsForwardSendFailures() { final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(17L); - valve.shutValve(17L); + valve.tryShutValve(17L); final CompletableFuture future = new CompletableFuture<>(); valve.sendEvent(sender.createSendAction(new TestOperatorEvent(), 10), future); From 4c730f57795f3e90bd285c0957725d800987250f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 13 Apr 2021 10:44:25 +0200 Subject: [PATCH 07/10] [hotfix][coordination] Reduce lambda nesting for action on CompletableFuture Directly use 'whenCompleteAsync()' with an executor, rather than 'whenComplete()' and nest a call to submit to the executor. --- .../OperatorCoordinatorHolder.java | 32 +++++++++---------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java index a6e2317b16df4..1baa73858111d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java @@ -276,23 +276,21 @@ private void checkpointCoordinatorInternal( final CompletableFuture coordinatorCheckpoint = new CompletableFuture<>(); - coordinatorCheckpoint.whenComplete( - (success, failure) -> - mainThreadExecutor.execute( - () -> { - if (failure != null) { - result.completeExceptionally(failure); - } else if (eventValve.tryShutValve(checkpointId)) { - result.complete(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")); - } - })); + coordinatorCheckpoint.whenCompleteAsync( + (success, failure) -> { + if (failure != null) { + result.completeExceptionally(failure); + } else if (eventValve.tryShutValve(checkpointId)) { + result.complete(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")); + } + }, + mainThreadExecutor); try { eventValve.markForCheckpoint(checkpointId); From 5ba44b6163acae687072ad19b32de5a9273d01a7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 14 Apr 2021 11:54:20 +0200 Subject: [PATCH 08/10] [hotfix][coordination] Remove unnecessary null check --- .../operators/coordination/OperatorCoordinatorHolder.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java index 1baa73858111d..8d96e156376a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java @@ -252,10 +252,7 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData } eventValve.openValveAndUnmarkCheckpoint(); - - if (context != null) { - context.resetFailed(); - } + context.resetFailed(); // when initial savepoints are restored, this call comes before the mainThreadExecutor // is available, which is needed to set up these gateways. So during the initial restore, From a8407aec8534af7ed761801a434a2b4467ed6d9a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 14 Apr 2021 17:34:16 +0200 Subject: [PATCH 09/10] [hotfix][tests] Minor debuggability improvements to CoordinatedSourceRescaleITCase --- .../source/reader/CoordinatedSourceRescaleITCase.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceRescaleITCase.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceRescaleITCase.java index 7ca15a324639c..f9e0e2714b603 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceRescaleITCase.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceRescaleITCase.java @@ -30,6 +30,7 @@ import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.util.TestLogger; import org.junit.Rule; import org.junit.Test; @@ -49,9 +50,11 @@ import static org.junit.Assert.assertThat; /** Tests if the coordinator handles up and downscaling. */ -public class CoordinatedSourceRescaleITCase { +public class CoordinatedSourceRescaleITCase extends TestLogger { + public static final String CREATED_CHECKPOINT = "successfully created checkpoint"; public static final String RESTORED_CHECKPOINT = "successfully restored checkpoint"; + @Rule public final TemporaryFolder temp = new TemporaryFolder(); @Test @@ -128,7 +131,8 @@ private StreamExecutionEnvironment createEnv( private static class FailingMapFunction extends RichMapFunction implements CheckpointListener { private static final long serialVersionUID = 699621912578369378L; - private boolean generateCheckpoint; + private final boolean generateCheckpoint; + private boolean processedRecord; FailingMapFunction(boolean generateCheckpoint) { this.generateCheckpoint = generateCheckpoint; @@ -136,6 +140,7 @@ private static class FailingMapFunction extends RichMapFunction @Override public Long map(Long value) throws Exception { + processedRecord = true; // run a bit before failing if (!generateCheckpoint && value % 100 == 42) { throw new Exception(RESTORED_CHECKPOINT); @@ -145,7 +150,7 @@ public Long map(Long value) throws Exception { @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { - if (generateCheckpoint && checkpointId > 5) { + if (generateCheckpoint && processedRecord && checkpointId > 5) { throw new Exception(CREATED_CHECKPOINT); } } From e42d8645d8603d0e93841b03a34ce80f2572e091 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 14 Apr 2021 05:02:55 +0200 Subject: [PATCH 10/10] [FLINK-21996][coordination] Ensure exactly-once guarantees for OperatorEvent RPCs This consists of two changes that work together: - Delay checkpoints until we have clarity about all in-flight OperatorEvents - Fail target subtask if the result future for an OperatorEvent send fails --- .../operators/coordination/EventSender.java | 37 ++++++++++ .../coordination/ExecutionSubtaskAccess.java | 10 +++ .../OperatorCoordinatorHolder.java | 72 ++++++++++++++++++- .../coordination/OperatorEventValve.java | 7 +- .../operators/coordination/SubtaskAccess.java | 11 +++ .../coordination/SubtaskGatewayImpl.java | 30 ++++++-- .../util/IncompleteFuturesTracker.java | 24 ++++--- .../coordination/EventReceivingTasks.java | 34 +++++---- .../OperatorCoordinatorHolderTest.java | 18 +++++ .../util/IncompleteFuturesTrackerTest.java | 18 ++++- .../OperatorEventSendingCheckpointITCase.java | 5 +- 11 files changed, 230 insertions(+), 36 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/EventSender.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/EventSender.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/EventSender.java new file mode 100644 index 0000000000000..c70a21ac97faa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/EventSender.java @@ -0,0 +1,37 @@ +/* + * 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.messages.Acknowledge; + +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; + +/** Simple interface for a component that takes and sends events. */ +@FunctionalInterface +interface EventSender { + + /** + * Takes the given Callable and calls it at a certain point to send the event. The result of + * that Callable are bridged to the given result future. + */ + void sendEvent( + Callable> sendAction, + CompletableFuture result); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java index fc363dec88e49..9babeb23b91bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java @@ -80,6 +80,11 @@ public ExecutionAttemptID currentAttempt() { return taskExecution.getAttemptId(); } + @Override + public String subtaskName() { + return taskExecution.getVertexWithAttempt(); + } + @Override public CompletableFuture hasSwitchedToRunning() { return taskExecution.getInitializingOrRunningFuture(); @@ -91,6 +96,11 @@ public boolean isStillRunning() { || taskExecution.getState() == ExecutionState.INITIALIZING; } + @Override + public void triggerTaskFailover(Throwable cause) { + taskExecution.fail(cause); + } + // ------------------------------------------------------------------------ static final class ExecutionJobVertexSubtaskAccess implements SubtaskAccessFactory { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java index 8d96e156376a7..a38ca167c20e8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java @@ -24,6 +24,8 @@ import org.apache.flink.runtime.concurrent.FutureUtils; 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.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; @@ -34,6 +36,8 @@ import javax.annotation.Nullable; +import java.util.Collection; +import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.function.Consumer; @@ -112,11 +116,15 @@ public class OperatorCoordinatorHolder implements OperatorCoordinatorCheckpointContext, AutoCloseable { + private static final Logger LOG = LoggerFactory.getLogger(OperatorCoordinatorHolder.class); + private final OperatorCoordinator coordinator; private final OperatorID operatorId; private final LazyInitializedCoordinatorContext context; private final SubtaskAccess.SubtaskAccessFactory taskAccesses; private final OperatorEventValve eventValve; + private final IncompleteFuturesTracker unconfirmedEvents; + private final EventSender eventSender; private final int operatorParallelism; private final int operatorMaxParallelism; @@ -139,7 +147,9 @@ private OperatorCoordinatorHolder( this.operatorParallelism = operatorParallelism; this.operatorMaxParallelism = operatorMaxParallelism; + this.unconfirmedEvents = new IncompleteFuturesTracker(); this.eventValve = new OperatorEventValve(); + this.eventSender = new ValveAndTrackerSender(eventValve, unconfirmedEvents); } public void lazyInitialize( @@ -278,7 +288,7 @@ private void checkpointCoordinatorInternal( if (failure != null) { result.completeExceptionally(failure); } else if (eventValve.tryShutValve(checkpointId)) { - result.complete(success); + completeCheckpointOnceEventsAreDone(checkpointId, result, success); } else { // if we cannot shut the valve, this means the checkpoint // has been aborted before, so the future is already @@ -299,6 +309,43 @@ private void checkpointCoordinatorInternal( } } + private void completeCheckpointOnceEventsAreDone( + final long checkpointId, + final CompletableFuture checkpointFuture, + final byte[] checkpointResult) { + + final Collection> pendingEvents = + unconfirmedEvents.getCurrentIncompleteAndReset(); + if (pendingEvents.isEmpty()) { + checkpointFuture.complete(checkpointResult); + return; + } + + LOG.info( + "Coordinator checkpoint {} for coordinator {} is awaiting {} pending events", + checkpointId, + operatorId, + pendingEvents.size()); + + final CompletableFuture conjunct = FutureUtils.waitForAll(pendingEvents); + conjunct.whenComplete( + (success, failure) -> { + if (failure == null) { + checkpointFuture.complete(checkpointResult); + } else { + // if we reach this situation, then anyways the checkpoint cannot + // complete because + // (a) the target task really is down + // (b) we have a potentially lost RPC message and need to + // do a task failover for the receiver to restore consistency + checkpointFuture.completeExceptionally( + new FlinkException( + "Failing OperatorCoordinator checkpoint because some OperatorEvents " + + "before this checkpoint barrier were not received by the target tasks.")); + } + }); + } + // ------------------------------------------------------------------------ // Checkpointing Callbacks // ------------------------------------------------------------------------ @@ -336,7 +383,7 @@ private void setupSubtaskGateway(int subtask) { final SubtaskAccess sta = taskAccesses.getAccessForSubtask(subtask); final OperatorCoordinator.SubtaskGateway gateway = - new SubtaskGatewayImpl(sta, eventValve, mainThreadExecutor); + new SubtaskGatewayImpl(sta, eventSender, mainThreadExecutor); // We need to do this synchronously here, otherwise we violate the contract that // 'subtaskFailed()' will never overtake 'subtaskReady()'. @@ -522,4 +569,25 @@ public ClassLoader getUserCodeClassloader() { return userCodeClassLoader; } } + + // ------------------------------------------------------------------------ + + private static final class ValveAndTrackerSender implements EventSender { + + private final OperatorEventValve valve; + private final IncompleteFuturesTracker tracker; + + ValveAndTrackerSender(OperatorEventValve valve, IncompleteFuturesTracker tracker) { + this.valve = valve; + this.tracker = tracker; + } + + @Override + public void sendEvent( + Callable> sendAction, + CompletableFuture result) { + valve.sendEvent(sendAction, result); + tracker.trackFutureWhileIncomplete(result); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java index 8e71e36ea6a12..7086dab4b6439 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorEventValve.java @@ -40,7 +40,7 @@ * that, one can register a "main thread executor" (as used by the mailbox components like RPC * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}. */ -final class OperatorEventValve { +final class OperatorEventValve implements EventSender { private static final long NO_CHECKPOINT = Long.MIN_VALUE; @@ -55,7 +55,7 @@ final class OperatorEventValve { @Nullable private ComponentMainThreadExecutor mainThreadExecutor; /** Constructs a new OperatorEventValve. */ - public OperatorEventValve() { + OperatorEventValve() { this.currentCheckpointId = NO_CHECKPOINT; this.lastCheckpointId = Long.MIN_VALUE; } @@ -82,6 +82,7 @@ public boolean isShut() { *

    This method makes no assumptions and gives no guarantees from which thread the result * future gets completed. */ + @Override public void sendEvent( Callable> sendAction, CompletableFuture result) { @@ -172,7 +173,7 @@ private void checkRunsInMainThread() { } } - private static void callSendAction( + private void callSendAction( Callable> sendAction, CompletableFuture result) { try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java index c654af860f321..333310cc2684d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java @@ -55,6 +55,12 @@ Callable> createEventSendAction( /** Gets the execution attempt ID of the attempt that this instance is bound to. */ ExecutionAttemptID currentAttempt(); + /** + * Gets a descriptive name of the operator's subtask , including name, subtask-id, parallelism, + * and execution attempt. + */ + String subtaskName(); + /** * The future returned here completes once the target subtask is in a running state. As running * state classify the states {@link ExecutionState#RUNNING} and {@link @@ -68,6 +74,11 @@ Callable> createEventSendAction( */ boolean isStillRunning(); + /** + * Triggers a failover for the subtaks execution attempt that this access instance is bound to. + */ + void triggerTaskFailover(Throwable cause); + // ------------------------------------------------------------------------ /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java index 0e8dade4e5310..11f33f5ab087d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java @@ -18,8 +18,10 @@ package org.apache.flink.runtime.operators.coordination; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.SerializedValue; @@ -34,14 +36,17 @@ */ 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 final SubtaskAccess subtaskAccess; - private final OperatorEventValve valve; + private final EventSender sender; private final Executor sendingExecutor; - SubtaskGatewayImpl( - SubtaskAccess subtaskAccess, OperatorEventValve valve, Executor sendingExecutor) { + SubtaskGatewayImpl(SubtaskAccess subtaskAccess, EventSender sender, Executor sendingExecutor) { this.subtaskAccess = subtaskAccess; - this.valve = valve; + this.sender = sender; this.sendingExecutor = sendingExecutor; } @@ -64,7 +69,22 @@ public CompletableFuture sendEvent(OperatorEvent evt) { subtaskAccess.createEventSendAction(serializedEvent); final CompletableFuture result = new CompletableFuture<>(); - sendingExecutor.execute(() -> valve.sendEvent(sendAction, result)); + FutureUtils.assertNoException( + result.handleAsync( + (success, failure) -> { + if (failure != null && subtaskAccess.isStillRunning()) { + String msg = + String.format( + EVENT_LOSS_ERROR_MESSAGE, + evt, + subtaskAccess.subtaskName()); + subtaskAccess.triggerTaskFailover(new FlinkException(msg)); + } + return null; + }, + sendingExecutor)); + + sendingExecutor.execute(() -> sender.sendEvent(sendAction, result)); return result; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java index 9904ce0008e25..1f8a9cdb04d2b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java @@ -18,15 +18,12 @@ package org.apache.flink.runtime.operators.coordination.util; -import org.apache.flink.annotation.VisibleForTesting; - import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.locks.ReentrantLock; @@ -64,6 +61,22 @@ public void trackFutureWhileIncomplete(CompletableFuture future) { future.whenComplete((success, failure) -> removeFromSet(future)); } + public Collection> getCurrentIncompleteAndReset() { + lock.lock(); + try { + if (incompleteFutures.isEmpty()) { + return Collections.emptySet(); + } + + final ArrayList> futures = new ArrayList<>(incompleteFutures); + incompleteFutures.clear(); + return futures; + + } finally { + lock.unlock(); + } + } + public void failAllFutures(Throwable cause) { final Collection> futuresToFail; @@ -95,9 +108,4 @@ void removeFromSet(CompletableFuture future) { lock.unlock(); } } - - @VisibleForTesting - Set> getTrackedFutures() { - return Collections.unmodifiableSet(incompleteFutures); - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java index baee4b10c20e3..dfed1eb617433 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java @@ -24,8 +24,6 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.util.SerializedValue; -import javax.annotation.Nullable; - import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -43,22 +41,27 @@ public class EventReceivingTasks implements SubtaskAccess.SubtaskAccessFactory { public static EventReceivingTasks createForNotYetRunningTasks() { - return new EventReceivingTasks(false, null); + return new EventReceivingTasks(false, CompletableFuture.completedFuture(Acknowledge.get())); } public static EventReceivingTasks createForRunningTasks() { - return new EventReceivingTasks(true, null); + return new EventReceivingTasks(true, CompletableFuture.completedFuture(Acknowledge.get())); } public static EventReceivingTasks createForRunningTasksFailingRpcs(Throwable rpcException) { - return new EventReceivingTasks(true, rpcException); + return new EventReceivingTasks(true, FutureUtils.completedExceptionally(rpcException)); + } + + public static EventReceivingTasks createForRunningTasksWithRpcResult( + CompletableFuture result) { + return new EventReceivingTasks(true, result); } // ------------------------------------------------------------------------ final ArrayList events = new ArrayList<>(); - @Nullable private final Throwable eventSendingFailureCause; + private final CompletableFuture eventSendingResult; private final Map subtasks = new HashMap<>(); @@ -66,9 +69,9 @@ public static EventReceivingTasks createForRunningTasksFailingRpcs(Throwable rpc private EventReceivingTasks( final boolean createdTasksAreRunning, - @Nullable final Throwable eventSendingFailureCause) { + final CompletableFuture eventSendingResult) { this.createdTasksAreRunning = createdTasksAreRunning; - this.eventSendingFailureCause = eventSendingFailureCause; + this.eventSendingResult = eventSendingResult; } // ------------------------------------------------------------------------ @@ -123,10 +126,7 @@ public void switchAllTasksToRunning() { Callable> createSendAction(OperatorEvent event, int subtask) { return () -> { events.add(new EventWithSubtask(event, subtask)); - - return eventSendingFailureCause == null - ? CompletableFuture.completedFuture(Acknowledge.get()) - : FutureUtils.completedExceptionally(eventSendingFailureCause); + return eventSendingResult; }; } @@ -206,6 +206,11 @@ public ExecutionAttemptID currentAttempt() { return executionAttemptId; } + @Override + public String subtaskName() { + return "test_task-" + subtaskIndex + " #: " + executionAttemptId; + } + @Override public CompletableFuture hasSwitchedToRunning() { return running; @@ -219,5 +224,10 @@ public boolean isStillRunning() { void switchToRunning() { running.complete(null); } + + @Override + public void triggerTaskFailover(Throwable cause) { + // ignore this in the tests + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java index 4eacd16d4a44f..3d3cddc5abab7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutorService; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks.EventWithSubtask; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -288,6 +289,23 @@ public void handleEventFromOperator(int subtask, OperatorEvent event) { globalFailure = null; } + @Test + public void checkpointCompletionWaitsForEventFutures() throws Exception { + final CompletableFuture ackFuture = new CompletableFuture<>(); + final EventReceivingTasks tasks = + EventReceivingTasks.createForRunningTasksWithRpcResult(ackFuture); + final OperatorCoordinatorHolder holder = + createCoordinatorHolder(tasks, TestingOperatorCoordinator::new); + + getCoordinator(holder).getSubtaskGateway(0).sendEvent(new TestOperatorEvent(0)); + + final CompletableFuture checkpointFuture = triggerAndCompleteCheckpoint(holder, 22L); + assertFalse(checkpointFuture.isDone()); + + ackFuture.complete(Acknowledge.get()); + assertTrue(checkpointFuture.isDone()); + } + /** * This test verifies that the order of Checkpoint Completion and Event Sending observed from * the outside matches that from within the OperatorCoordinator. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java index e9942888203ba..a2aff7fd87afd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java @@ -24,6 +24,7 @@ import java.util.concurrent.ExecutionException; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertThat; @@ -40,7 +41,7 @@ public void testFutureTracked() { tracker.trackFutureWhileIncomplete(future); - assertThat(tracker.getTrackedFutures(), contains(future)); + assertThat(tracker.getCurrentIncompleteAndReset(), contains(future)); } @Test @@ -51,7 +52,7 @@ public void testFutureRemovedAfterCompletion() { tracker.trackFutureWhileIncomplete(future); future.complete(null); - assertThat(tracker.getTrackedFutures(), not(contains(future))); + assertThat(tracker.getCurrentIncompleteAndReset(), not(contains(future))); } @Test @@ -62,7 +63,7 @@ public void testFutureNotAddedIfAlreadyCompleted() { future.complete(null); tracker.trackFutureWhileIncomplete(future); - assertThat(tracker.getTrackedFutures(), not(contains(future))); + assertThat(tracker.getCurrentIncompleteAndReset(), not(contains(future))); } @Test @@ -102,4 +103,15 @@ public void testFailFuturesImmediately() throws Exception { assertSame(expectedException, e.getCause()); } } + + @Test + public void testResetClearsTrackedFutures() { + final IncompleteFuturesTracker tracker = new IncompleteFuturesTracker(); + + final CompletableFuture future = new CompletableFuture<>(); + tracker.trackFutureWhileIncomplete(future); + tracker.getCurrentIncompleteAndReset(); + + assertThat(tracker.getCurrentIncompleteAndReset(), empty()); + } } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventSendingCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventSendingCheckpointITCase.java index bd59ab0355742..9ca560ce145c7 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventSendingCheckpointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorEventSendingCheckpointITCase.java @@ -53,7 +53,6 @@ import akka.actor.ActorSystem; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import javax.annotation.Nullable; @@ -107,7 +106,6 @@ public static void clearEnvAndStopMiniCluster() throws Exception { * event was lost and trigger recovery to prevent data loss. Data loss would manifest in a * stalled test, because we could wait forever to collect the required number of events back. */ - @Ignore // ignore for now, because this test fails due to FLINK-21996 @Test public void testOperatorEventLostNoReaderFailure() throws Exception { final int[] eventsToLose = new int[] {2, 4, 6}; @@ -125,7 +123,6 @@ public void testOperatorEventLostNoReaderFailure() throws Exception { * (which is after the second successful event delivery, the fourth event), there is * additionally a failure on the reader that triggers recovery. */ - @Ignore // ignore for now, because this test fails due to FLINK-21996 @Test public void testOperatorEventLostWithReaderFailure() throws Exception { final int[] eventsToLose = new int[] {1, 3}; @@ -224,6 +221,8 @@ public Long map(Long value) throws Exception { }); final List sequence = numbers.executeAndCollect(numElements); + // the recovery may change the order of splits, so the sequence might be out-of-order + sequence.sort(Long::compareTo); final List expectedSequence = LongStream.rangeClosed(1L, numElements).boxed().collect(Collectors.toList());