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); } } 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..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 @@ -854,6 +879,8 @@ private void triggerCheckpointHelper( */ public CompletableFuture sendOperatorEvent( OperatorID operatorId, SerializedValue event) { + + assertRunningInJobMasterMainThread(); final LogicalSlot slot = assignedResource; if (slot != null && (getState() == RUNNING || getState() == INITIALIZING)) { @@ -1426,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); } 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 new file mode 100644 index 0000000000000..9babeb23b91bd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/ExecutionSubtaskAccess.java @@ -0,0 +1,128 @@ +/* + * 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 String subtaskName() { + return taskExecution.getVertexWithAttempt(); + } + + @Override + public CompletableFuture hasSwitchedToRunning() { + return taskExecution.getInitializingOrRunningFuture(); + } + + @Override + public boolean isStillRunning() { + return taskExecution.getState() == ExecutionState.RUNNING + || taskExecution.getState() == ExecutionState.INITIALIZING; + } + + @Override + public void triggerTaskFailover(Throwable cause) { + taskExecution.fail(cause); + } + + // ------------------------------------------------------------------------ + + 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 21974c361f476..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 @@ -21,13 +21,13 @@ 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.runtime.operators.coordination.util.IncompleteFuturesTracker; 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 +36,10 @@ import javax.annotation.Nullable; -import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.Callable; 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; @@ -114,12 +114,17 @@ * 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 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; @@ -131,25 +136,33 @@ 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.unconfirmedEvents = new IncompleteFuturesTracker(); + this.eventValve = new OperatorEventValve(); + this.eventSender = new ValveAndTrackerSender(eventValve, unconfirmedEvents); } public void lazyInitialize( Consumer globalFailureHandler, ComponentMainThreadExecutor mainThreadExecutor) { + this.globalFailureHandler = globalFailureHandler; this.mainThreadExecutor = mainThreadExecutor; + eventValve.setMainThreadExecutorForValidation(mainThreadExecutor); context.lazyInitialize(globalFailureHandler, mainThreadExecutor); + + setupAllSubtaskGateways(); } // ------------------------------------------------------------------------ @@ -179,7 +192,6 @@ public int currentParallelism() { // OperatorCoordinator Interface // ------------------------------------------------------------------------ - @Override public void start() throws Exception { mainThreadExecutor.assertRunningInMainThread(); checkState(context.isInitialized(), "Coordinator Context is not yet initialized"); @@ -192,22 +204,24 @@ 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); - 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); } @@ -247,10 +261,19 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData mainThreadExecutor.assertRunningInMainThread(); } - eventValve.reset(); - if (context != null) { - context.resetFailed(); + eventValve.openValveAndUnmarkCheckpoint(); + 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); } @@ -264,13 +287,14 @@ private void checkpointCoordinatorInternal( (success, failure) -> { if (failure != null) { result.completeExceptionally(failure); + } else if (eventValve.tryShutValve(checkpointId)) { + completeCheckpointOnceEventsAreDone(checkpointId, result, 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")); } }, mainThreadExecutor); @@ -285,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 // ------------------------------------------------------------------------ @@ -307,6 +368,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, eventSender, 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 // ------------------------------------------------------------------------ @@ -322,24 +428,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); } } @@ -347,25 +446,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); } // ------------------------------------------------------------------------ @@ -388,7 +488,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; @@ -400,12 +499,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; @@ -438,33 +535,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(); @@ -499,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 99eb7aaea1247..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 @@ -21,38 +21,30 @@ 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.FlinkException; -import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.ExceptionUtils; 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 * components) via {@link #setMainThreadExecutorForValidation(ComponentMainThreadExecutor)}. */ -final class OperatorEventValve { +final class OperatorEventValve implements EventSender { 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; @@ -62,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. */ + OperatorEventValve() { this.currentCheckpointId = NO_CHECKPOINT; this.lastCheckpointId = Long.MIN_VALUE; } @@ -92,28 +77,22 @@ 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. */ + @Override public void sendEvent( - SerializedValue event, - int subtask, + Callable> sendAction, CompletableFuture result) { checkRunsInMainThread(); - if (!shut) { - final CompletableFuture ack = eventSender.apply(event, subtask); - FutureUtils.forward(ack, result); - return; + if (shut) { + blockedEvents.add(new BlockedEvent(sendAction, result)); + } else { + callSendAction(sendAction, result); } - - final List eventsForTask = - blockedEvents.computeIfAbsent(subtask, (key) -> new ArrayList<>()); - eventsForTask.add(new BlockedEvent(event, subtask, result)); } /** @@ -150,21 +129,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) { @@ -188,52 +160,11 @@ public void openValveAndUnmarkCheckpoint() { return; } - for (List eventsForTask : blockedEvents.values()) { - for (BlockedEvent blockedEvent : eventsForTask) { - final CompletableFuture ackFuture = - eventSender.apply(blockedEvent.event, blockedEvent.subtask); - FutureUtils.forward(ackFuture, 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); - } + for (BlockedEvent blockedEvent : blockedEvents) { + callSendAction(blockedEvent.sendAction, blockedEvent.future); } 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() { @@ -242,21 +173,30 @@ private void checkRunsInMainThread() { } } + private 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; } } } 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..333310cc2684d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskAccess.java @@ -0,0 +1,98 @@ +/* + * 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(); + + /** + * 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 + * ExecutionState#INITIALIZING}. + */ + CompletableFuture hasSwitchedToRunning(); + + /** + * Checks whether the execution is still in a running state. See {@link #hasSwitchedToRunning()} + * for details. + */ + boolean isStillRunning(); + + /** + * Triggers a failover for the subtaks execution attempt that this access instance is bound to. + */ + void triggerTaskFailover(Throwable cause); + + // ------------------------------------------------------------------------ + + /** + * 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..11f33f5ab087d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java @@ -0,0 +1,104 @@ +/* + * 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.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; + +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 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 EventSender sender; + private final Executor sendingExecutor; + + SubtaskGatewayImpl(SubtaskAccess subtaskAccess, EventSender sender, Executor sendingExecutor) { + this.subtaskAccess = subtaskAccess; + this.sender = sender; + 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<>(); + 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; + } + + @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..1f8a9cdb04d2b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTracker.java @@ -0,0 +1,111 @@ +/* + * 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 javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +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 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; + + 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(); + } + } +} 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); } } 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..dfed1eb617433 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/EventReceivingTasks.java @@ -0,0 +1,233 @@ +/* + * 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 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, CompletableFuture.completedFuture(Acknowledge.get())); + } + + public static EventReceivingTasks createForRunningTasks() { + return new EventReceivingTasks(true, CompletableFuture.completedFuture(Acknowledge.get())); + } + + public static EventReceivingTasks createForRunningTasksFailingRpcs(Throwable rpcException) { + return new EventReceivingTasks(true, FutureUtils.completedExceptionally(rpcException)); + } + + public static EventReceivingTasks createForRunningTasksWithRpcResult( + CompletableFuture result) { + return new EventReceivingTasks(true, result); + } + + // ------------------------------------------------------------------------ + + final ArrayList events = new ArrayList<>(); + + private final CompletableFuture eventSendingResult; + + private final Map subtasks = new HashMap<>(); + + private final boolean createdTasksAreRunning; + + private EventReceivingTasks( + final boolean createdTasksAreRunning, + final CompletableFuture eventSendingResult) { + this.createdTasksAreRunning = createdTasksAreRunning; + this.eventSendingResult = eventSendingResult; + } + + // ------------------------------------------------------------------------ + // 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 eventSendingResult; + }; + } + + // ------------------------------------------------------------------------ + + /** 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 String subtaskName() { + return "test_task-" + subtaskIndex + " #: " + executionAttemptId; + } + + @Override + public CompletableFuture hasSwitchedToRunning() { + return running; + } + + @Override + public boolean isStillRunning() { + return true; + } + + 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/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..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 @@ -23,9 +23,8 @@ 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 +39,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 +72,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 +84,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 +100,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))); - } + getCoordinator(holder).getSubtaskGateway(1).sendEvent(new TestOperatorEvent(999)); - @Test - public void restoreDropsBlockedEvents() 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.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 +179,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 +202,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 +264,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); @@ -320,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. @@ -355,7 +341,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 +363,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 +400,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 +434,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 +494,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 +525,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 +542,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 +571,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..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 @@ -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,139 +35,99 @@ 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); + @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(); + + final boolean shut = valve.tryShutValve(123L); - valve.shutValve(123L); + assertFalse(shut); } - @Test(expected = IllegalStateException.class) - public void errorShuttingValveForOtherMark() throws Exception { - final TestEventSender sender = new TestEventSender(); - final OperatorEventValve valve = new OperatorEventValve(sender); + @Test + public void notShuttingValveForOtherMark() { + final OperatorEventValve valve = new OperatorEventValve(); valve.markForCheckpoint(100L); - valve.shutValve(123L); + final boolean shut = valve.tryShutValve(123L); + + assertFalse(shut); } @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); + valve.tryShutValve(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); + valve.tryShutValve(17L); 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); + valve.tryShutValve(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..a2aff7fd87afd --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/util/IncompleteFuturesTrackerTest.java @@ -0,0 +1,117 @@ +/* + * 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.empty; +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.getCurrentIncompleteAndReset(), contains(future)); + } + + @Test + public void testFutureRemovedAfterCompletion() { + final IncompleteFuturesTracker tracker = new IncompleteFuturesTracker(); + final CompletableFuture future = new CompletableFuture<>(); + + tracker.trackFutureWhileIncomplete(future); + future.complete(null); + + assertThat(tracker.getCurrentIncompleteAndReset(), 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.getCurrentIncompleteAndReset(), 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()); + } + } + + @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-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 { 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());