diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index 71c5e77cfa6d7..8dc504a46e9f3 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -111,9 +111,9 @@ public class RocksDBAsyncSnapshotTest extends TestLogger { @Test public void testFullyAsyncSnapshot() throws Exception { - final OneInputStreamTask task = new OneInputStreamTask<>(); - - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>(task, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); testHarness.configureForKeyedStream(new KeySelector() { @@ -179,6 +179,8 @@ public void acknowledgeCheckpoint( testHarness.invoke(mockEnv); + final OneInputStreamTask task = testHarness.getTask(); + // wait for the task to be running for (Field field: StreamTask.class.getDeclaredFields()) { if (field.getName().equals("isRunning")) { @@ -213,13 +215,13 @@ public void acknowledgeCheckpoint( /** * This tests ensures that canceling of asynchronous snapshots works as expected and does not block. - * @throws Exception */ @Test public void testCancelFullyAsyncCheckpoints() throws Exception { - final OneInputStreamTask task = new OneInputStreamTask<>(); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>(task, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); testHarness.configureForKeyedStream(new KeySelector() { @@ -278,6 +280,8 @@ public MemCheckpointStreamFactory.MemoryCheckpointOutputStream createCheckpointS testHarness.invoke(mockEnv); + final OneInputStreamTask task = testHarness.getTask(); + // wait for the task to be running for (Field field: StreamTask.class.getDeclaredFields()) { if (field.getName().equals("isRunning")) { diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java index c74db288af923..c79e3d7bf8e99 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java @@ -21,7 +21,9 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.metrics.jmx.JMXReporter; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -120,24 +122,20 @@ public void testJobManagerJMXMetricAccess() throws Exception { * Utility to block/unblock a task. */ public static class BlockingInvokable extends AbstractInvokable { - private static boolean blocking = true; - private static final Object lock = new Object(); + + private static final OneShotLatch LATCH = new OneShotLatch(); + + public BlockingInvokable(Environment environment) { + super(environment); + } @Override public void invoke() throws Exception { - while (blocking) { - synchronized (lock) { - lock.wait(); - } - } + LATCH.await(); } public static void unblock() { - blocking = false; - - synchronized (lock) { - lock.notifyAll(); - } + LATCH.trigger(); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java index 27e5d0cdfe5fb..813c292b2fc88 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointOptions.java @@ -18,19 +18,20 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; + import static org.apache.flink.util.Preconditions.checkNotNull; import java.io.Serializable; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; /** * Options for performing the checkpoint. * *

The {@link CheckpointProperties} are related and cover properties that * are only relevant at the {@link CheckpointCoordinator}. These options are - * relevant at the {@link StatefulTask} instances running on task managers. + * relevant at the {@link AbstractInvokable} instances running on task managers. */ public class CheckpointOptions implements Serializable { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java index a36fc57a91321..19eb3d18e56cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java @@ -66,6 +66,7 @@ */ public abstract class AbstractIterativeTask extends BatchTask implements Terminable { + private static final Logger log = LoggerFactory.getLogger(AbstractIterativeTask.class); protected LongSumAggregator worksetAggregator; @@ -86,6 +87,17 @@ public abstract class AbstractIterativeTask extends Batc private volatile boolean terminationRequested; + // -------------------------------------------------------------------------------------------- + + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public AbstractIterativeTask(Environment environment) { + super(environment); + } + // -------------------------------------------------------------------------------------------- // Main life cycle methods that implement the iterative behavior // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java index 2a95a65111139..1dd3da4a29926 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.disk.InputViewIterator; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent; @@ -100,6 +101,17 @@ public class IterationHeadTask extends AbstractIte // -------------------------------------------------------------------------------------------- + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public IterationHeadTask(Environment environment) { + super(environment); + } + + // -------------------------------------------------------------------------------------------- + @Override protected int getNumTaskInputs() { // this task has an additional input in the workset case for the initial solution set diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java index c5fd1334bbfb3..d1bd5dba68bf1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.iterative.task; import org.apache.flink.api.common.functions.Function; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel; @@ -49,6 +50,19 @@ public class IterationIntermediateTask extends AbstractI private WorksetUpdateOutputCollector worksetUpdateOutputCollector; + // -------------------------------------------------------------------------------------------- + + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public IterationIntermediateTask(Environment environment) { + super(environment); + } + + // -------------------------------------------------------------------------------------------- + @Override protected void initialize() throws Exception { super.initialize(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java index 6a38fcce39915..14f3612255cd0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.aggregators.AggregatorWithName; import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.runtime.event.TaskEvent; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.reader.MutableRecordReader; import org.apache.flink.runtime.iterative.event.AllWorkersDoneEvent; import org.apache.flink.runtime.iterative.event.TerminationEvent; @@ -73,6 +74,17 @@ public class IterationSynchronizationSinkTask extends AbstractInvokable implemen // -------------------------------------------------------------------------------------------- + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public IterationSynchronizationSinkTask(Environment environment) { + super(environment); + } + + // -------------------------------------------------------------------------------------------- + @Override public void invoke() throws Exception { this.headEventReader = new MutableRecordReader<>( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java index 3ec3a8e13972e..7217bfe152a97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.iterative.task; import org.apache.flink.api.common.functions.Function; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrier; import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrierBroker; import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatch; @@ -47,6 +48,19 @@ public class IterationTailTask extends AbstractIterative private WorksetUpdateOutputCollector worksetUpdateOutputCollector; + // -------------------------------------------------------------------------------------------- + + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public IterationTailTask(Environment environment) { + super(environment); + } + + // -------------------------------------------------------------------------------------------- + @Override protected void initialize() throws Exception { super.initialize(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java index f63a762ffcd85..1734d68583fb4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java @@ -20,22 +20,53 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.operators.BatchTask; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** - * This is the abstract base class for every task that can be executed by a - * TaskManager. Concrete tasks like the vertices of batch jobs (see - * {@link BatchTask} inherit from this class. + * This is the abstract base class for every task that can be executed by a TaskManager. + * Concrete tasks extend this class, for example the streaming and batch tasks. * *

The TaskManager invokes the {@link #invoke()} method when executing a * task. All operations of the task happen in this method (setting up input * output stream readers and writers as well as the task's core operation). + * + *

All classes that extend must offer a constructor {@code MyTask(Environment, TaskStateSnapshot)}. + * Tasks that are always stateless can, for convenience, also only implement the constructor + * {@code MyTask(Environment)}. + * + *

Developer note: While constructors cannot be enforced at compile time, we did not yet venture + * on the endeavor of introducing factories (it is only an internal API after all, and with Java 8, + * one can use {@code Class::new} almost like a factory lambda. + * + *

NOTE: There is no constructor that accepts and initial task state snapshot + * and stores it in a variable. That is on purpose, because the AbstractInvokable itself + * does not need the state snapshot (only subclasses such as StreamTask do need the state) + * and we do not want to store a reference indefinitely, thus preventing cleanup of + * the initial state structure by the Garbage Collector. + * + *

Any subclass that supports recoverable state and participates in + * checkpointing needs to override {@link #triggerCheckpoint(CheckpointMetaData, CheckpointOptions)}, + * {@link #triggerCheckpointOnBarrier(CheckpointMetaData, CheckpointOptions, CheckpointMetrics)}, + * {@link #abortCheckpointOnBarrier(long, Throwable)} and {@link #notifyCheckpointComplete(long)}. */ public abstract class AbstractInvokable { /** The environment assigned to this invokable. */ - private Environment environment; + private final Environment environment; + + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public AbstractInvokable(Environment environment) { + this.environment = checkNotNull(environment); + } /** * Starts the execution. @@ -46,7 +77,7 @@ public abstract class AbstractInvokable { * *

All resources should be cleaned up when the method returns. Make sure * to guard the code with try-finally blocks where necessary. - * + * * @throws Exception * Tasks may forward their exceptions for the TaskManager to handle through failure/recovery. */ @@ -62,16 +93,6 @@ public abstract class AbstractInvokable { public void cancel() throws Exception { // The default implementation does nothing. } - - /** - * Sets the environment of this task. - * - * @param environment - * the environment of this task - */ - public final void setEnvironment(Environment environment) { - this.environment = environment; - } /** * Returns the environment of this task. @@ -133,4 +154,65 @@ public Configuration getJobConfiguration() { public ExecutionConfig getExecutionConfig() { return this.environment.getExecutionConfig(); } + + // ------------------------------------------------------------------------ + // Checkpointing Methods + // ------------------------------------------------------------------------ + + /** + * This method is called to trigger a checkpoint, asynchronously by the checkpoint + * coordinator. + * + *

This method is called for tasks that start the checkpoints by injecting the initial barriers, + * i.e., the source tasks. In contrast, checkpoints on downstream operators, which are the result of + * receiving checkpoint barriers, invoke the {@link #triggerCheckpointOnBarrier(CheckpointMetaData, CheckpointOptions, CheckpointMetrics)} + * method. + * + * @param checkpointMetaData Meta data for about this checkpoint + * @param checkpointOptions Options for performing this checkpoint + * + * @return {@code false} if the checkpoint can not be carried out, {@code true} otherwise + */ + public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) throws Exception { + throw new UnsupportedOperationException(String.format("triggerCheckpoint not supported by %s", this.getClass().getName())); + } + + /** + * This method is called when a checkpoint is triggered as a result of receiving checkpoint + * barriers on all input streams. + * + * @param checkpointMetaData Meta data for about this checkpoint + * @param checkpointOptions Options for performing this checkpoint + * @param checkpointMetrics Metrics about this checkpoint + * + * @throws Exception Exceptions thrown as the result of triggering a checkpoint are forwarded. + */ + public void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics checkpointMetrics) throws Exception { + throw new UnsupportedOperationException(String.format("triggerCheckpointOnBarrier not supported by %s", this.getClass().getName())); + } + + /** + * Aborts a checkpoint as the result of receiving possibly some checkpoint barriers, + * but at least one {@link org.apache.flink.runtime.io.network.api.CancelCheckpointMarker}. + * + *

This requires implementing tasks to forward a + * {@link org.apache.flink.runtime.io.network.api.CancelCheckpointMarker} to their outputs. + * + * @param checkpointId The ID of the checkpoint to be aborted. + * @param cause The reason why the checkpoint was aborted during alignment + */ + public void abortCheckpointOnBarrier(long checkpointId, Throwable cause) throws Exception { + throw new UnsupportedOperationException(String.format("abortCheckpointOnBarrier not supported by %s", this.getClass().getName())); + } + + /** + * Invoked when a checkpoint has been completed, i.e., when the checkpoint coordinator has received + * the notification from all participating tasks. + * + * @param checkpointId The ID of the checkpoint that is complete.. + * @throws Exception The notification method may forward its exceptions. + */ + public void notifyCheckpointComplete(long checkpointId) throws Exception { + throw new UnsupportedOperationException(String.format("notifyCheckpointComplete not supported by %s", this.getClass().getName())); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StatefulTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StatefulTask.java deleted file mode 100644 index 00db01ffd2e04..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StatefulTask.java +++ /dev/null @@ -1,88 +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.jobgraph.tasks; - -import org.apache.flink.runtime.checkpoint.CheckpointMetaData; -import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; - -/** - * This interface must be implemented by any invokable that has recoverable state and participates - * in checkpointing. - */ -public interface StatefulTask { - - /** - * Sets the initial state of the operator, upon recovery. The initial state is typically - * a snapshot of the state from a previous execution. - * - * @param taskStateHandles All state handle for the task. - */ - void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception; - - /** - * This method is called to trigger a checkpoint, asynchronously by the checkpoint - * coordinator. - * - *

This method is called for tasks that start the checkpoints by injecting the initial barriers, - * i.e., the source tasks. In contrast, checkpoints on downstream operators, which are the result of - * receiving checkpoint barriers, invoke the - * {@link #triggerCheckpointOnBarrier(CheckpointMetaData, CheckpointOptions, CheckpointMetrics)} method. - * - * @param checkpointMetaData Meta data for about this checkpoint - * @param checkpointOptions Options for performing this checkpoint - * - * @return {@code false} if the checkpoint can not be carried out, {@code true} otherwise - */ - boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) throws Exception; - - /** - * This method is called when a checkpoint is triggered as a result of receiving checkpoint - * barriers on all input streams. - * - * @param checkpointMetaData Meta data for about this checkpoint - * @param checkpointOptions Options for performing this checkpoint - * @param checkpointMetrics Metrics about this checkpoint - * - * @throws Exception Exceptions thrown as the result of triggering a checkpoint are forwarded. - */ - void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics checkpointMetrics) throws Exception; - - /** - * Aborts a checkpoint as the result of receiving possibly some checkpoint barriers, - * but at least one {@link org.apache.flink.runtime.io.network.api.CancelCheckpointMarker}. - * - *

This requires implementing tasks to forward a - * {@link org.apache.flink.runtime.io.network.api.CancelCheckpointMarker} to their outputs. - * - * @param checkpointId The ID of the checkpoint to be aborted. - * @param cause The reason why the checkpoint was aborted during alignment - */ - void abortCheckpointOnBarrier(long checkpointId, Throwable cause) throws Exception; - - /** - * Invoked when a checkpoint has been completed, i.e., when the checkpoint coordinator has received - * the notification from all participating tasks. - * - * @param checkpointId The ID of the checkpoint that is complete.. - * @throws Exception The notification method may forward its exceptions. - */ - void notifyCheckpointComplete(long checkpointId) throws Exception; -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java index be81877e734ce..e6978697b471f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -79,7 +79,7 @@ public class BatchTask extends AbstractInvokable implements TaskContext { protected static final Logger LOG = LoggerFactory.getLogger(BatchTask.class); - + // -------------------------------------------------------------------------------------------- /** @@ -215,6 +215,19 @@ public class BatchTask extends AbstractInvokable impleme protected Map> accumulatorMap; private OperatorMetricGroup metrics; + // -------------------------------------------------------------------------------------------- + // Constructor + // -------------------------------------------------------------------------------------------- + + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public BatchTask(Environment environment) { + super(environment); + } + // -------------------------------------------------------------------------------------------- // Task Interface // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index bb253abb561e2..0ea376e8c4de5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -46,6 +46,7 @@ import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.util.MutableObjectIterator; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,12 +56,12 @@ * @see OutputFormat */ public class DataSinkTask extends AbstractInvokable { - + // Obtain DataSinkTask Logger private static final Logger LOG = LoggerFactory.getLogger(DataSinkTask.class); // -------------------------------------------------------------------------------------------- - + // OutputFormat instance. volatile, because the asynchronous canceller may access it private volatile OutputFormat format; @@ -83,6 +84,15 @@ public class DataSinkTask extends AbstractInvokable { private volatile boolean cleanupCalled; + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public DataSinkTask(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { // -------------------------------------------------------------------- @@ -298,7 +308,7 @@ public void cancel() throws Exception { LOG.debug(getLogString("Cancelling data sink operator")); } - + /** * Initializes the OutputFormat implementation and configuration. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index 14378770d8b97..cdfe1fa12f913 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -42,6 +42,7 @@ import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.operators.util.metrics.CountingCollector; import org.apache.flink.util.Collector; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,6 +81,15 @@ public class DataSourceTask extends AbstractInvokable { // cancel flag private volatile boolean taskCanceled = false; + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public DataSourceTask(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { // -------------------------------------------------------------------- @@ -244,7 +254,7 @@ public void cancel() throws Exception { this.taskCanceled = true; LOG.debug(getLogString("Cancelling data source operator")); } - + /** * Initializes the InputFormat implementation and configuration. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 3c1e98e4e1b9f..16437d91ca2cc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -36,7 +36,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; -import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotCheckpointingException; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotReadyException; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -62,13 +61,13 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.jobgraph.tasks.StoppableTask; import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.WrappingRuntimeException; @@ -80,6 +79,8 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.net.URL; import java.util.Collection; import java.util.HashMap; @@ -589,9 +590,6 @@ else if (current == ExecutionState.CANCELING) { taskCancellationTimeout = executionConfig.getTaskCancellationTimeout(); } - // now load the task's invokable code - invokable = loadAndInstantiateInvokable(userCodeClassLoader, nameOfInvokableClass); - if (isCanceledOrFailed()) { throw new CancelTaskException(); } @@ -653,8 +651,7 @@ else if (current == ExecutionState.CANCELING) { // call the user code initialization methods // ---------------------------------------------------------------- - TaskKvStateRegistry kvStateRegistry = network - .createKvStateTaskRegistry(jobId, getJobVertexId()); + TaskKvStateRegistry kvStateRegistry = network.createKvStateTaskRegistry(jobId, getJobVertexId()); Environment env = new RuntimeEnvironment( jobId, @@ -680,25 +677,8 @@ else if (current == ExecutionState.CANCELING) { metrics, this); - // let the task code create its readers and writers - invokable.setEnvironment(env); - - // the very last thing before the actual execution starts running is to inject - // the state into the task. the state is non-empty if this is an execution - // of a task that failed but had backuped state from a checkpoint - - if (null != taskStateHandles) { - if (invokable instanceof StatefulTask) { - StatefulTask op = (StatefulTask) invokable; - op.setInitialState(taskStateHandles); - } else { - throw new IllegalStateException("Found operator state for a non-stateful task invokable"); - } - // be memory and GC friendly - since the code stays in invoke() for a potentially long time, - // we clear the reference to the state handle - //noinspection UnusedAssignment - taskStateHandles = null; - } + // now load and instantiate the task's invokable code + invokable = loadAndInstantiateInvokable(userCodeClassLoader, nameOfInvokableClass, env, taskStateHandles); // ---------------------------------------------------------------- // actual task core work @@ -893,23 +873,6 @@ private ClassLoader createUserCodeClassloader() throws Exception { return userCodeClassLoader; } - private AbstractInvokable loadAndInstantiateInvokable(ClassLoader classLoader, String className) throws Exception { - Class invokableClass; - try { - invokableClass = Class.forName(className, true, classLoader) - .asSubclass(AbstractInvokable.class); - } - catch (Throwable t) { - throw new Exception("Could not load the task's invokable class.", t); - } - try { - return invokableClass.newInstance(); - } - catch (Throwable t) { - throw new Exception("Could not instantiate the task's invokable class.", t); - } - } - private void removeCachedFiles(Map> entries, FileCache fileCache) { // cancel and release all distributed cache files try { @@ -991,8 +954,8 @@ public void stopExecution() throws UnsupportedOperationException { @Override public void run() { try { - ((StoppableTask)invokable).stop(); - } catch(RuntimeException e) { + ((StoppableTask) invokable).stop(); + } catch (RuntimeException e) { LOG.error("Stopping task {} ({}) failed.", taskNameWithSubtask, executionId, e); taskManagerActions.failTask(executionId, e); } @@ -1172,8 +1135,7 @@ public void triggerPartitionProducerStateCheck( // ------------------------------------------------------------------------ /** - * Calls the invokable to trigger a checkpoint, if the invokable implements the interface - * {@link StatefulTask}. + * Calls the invokable to trigger a checkpoint. * * @param checkpointID The ID identifying the checkpoint. * @param checkpointTimestamp The timestamp associated with the checkpoint. @@ -1188,52 +1150,43 @@ public void triggerCheckpointBarrier( final CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointID, checkpointTimestamp); if (executionState == ExecutionState.RUNNING && invokable != null) { - if (invokable instanceof StatefulTask) { - // build a local closure - final StatefulTask statefulTask = (StatefulTask) invokable; - final String taskName = taskNameWithSubtask; - final SafetyNetCloseableRegistry safetyNetCloseableRegistry = - FileSystemSafetyNet.getSafetyNetCloseableRegistryForThread(); - Runnable runnable = new Runnable() { - @Override - public void run() { - // set safety net from the task's context for checkpointing thread - LOG.debug("Creating FileSystem stream leak safety net for {}", Thread.currentThread().getName()); - FileSystemSafetyNet.setSafetyNetCloseableRegistryForThread(safetyNetCloseableRegistry); - try { - boolean success = statefulTask.triggerCheckpoint(checkpointMetaData, checkpointOptions); - if (!success) { - checkpointResponder.declineCheckpoint( - getJobID(), getExecutionId(), checkpointID, - new CheckpointDeclineTaskNotReadyException(taskName)); - } + // build a local closure + final String taskName = taskNameWithSubtask; + final SafetyNetCloseableRegistry safetyNetCloseableRegistry = + FileSystemSafetyNet.getSafetyNetCloseableRegistryForThread(); + + Runnable runnable = new Runnable() { + @Override + public void run() { + // set safety net from the task's context for checkpointing thread + LOG.debug("Creating FileSystem stream leak safety net for {}", Thread.currentThread().getName()); + FileSystemSafetyNet.setSafetyNetCloseableRegistryForThread(safetyNetCloseableRegistry); + + try { + boolean success = invokable.triggerCheckpoint(checkpointMetaData, checkpointOptions); + if (!success) { + checkpointResponder.declineCheckpoint( + getJobID(), getExecutionId(), checkpointID, + new CheckpointDeclineTaskNotReadyException(taskName)); } - catch (Throwable t) { - if (getExecutionState() == ExecutionState.RUNNING) { - failExternally(new Exception( - "Error while triggering checkpoint " + checkpointID + " for " + - taskNameWithSubtask, t)); - } else { - LOG.debug("Encountered error while triggering checkpoint {} for " + - "{} ({}) while being not in state running.", checkpointID, - taskNameWithSubtask, executionId, t); - } - } finally { - FileSystemSafetyNet.setSafetyNetCloseableRegistryForThread(null); + } + catch (Throwable t) { + if (getExecutionState() == ExecutionState.RUNNING) { + failExternally(new Exception( + "Error while triggering checkpoint " + checkpointID + " for " + + taskNameWithSubtask, t)); + } else { + LOG.debug("Encountered error while triggering checkpoint {} for " + + "{} ({}) while being not in state running.", checkpointID, + taskNameWithSubtask, executionId, t); } + } finally { + FileSystemSafetyNet.setSafetyNetCloseableRegistryForThread(null); } - }; - executeAsyncCallRunnable(runnable, String.format("Checkpoint Trigger for %s (%s).", taskNameWithSubtask, executionId)); - } - else { - checkpointResponder.declineCheckpoint(jobId, executionId, checkpointID, - new CheckpointDeclineTaskNotCheckpointingException(taskNameWithSubtask)); - - LOG.error("Task received a checkpoint request, but is not a checkpointing task - {} ({}).", - taskNameWithSubtask, executionId); - - } + } + }; + executeAsyncCallRunnable(runnable, String.format("Checkpoint Trigger for %s (%s).", taskNameWithSubtask, executionId)); } else { LOG.debug("Declining checkpoint request for non-running task {} ({}).", taskNameWithSubtask, executionId); @@ -1245,37 +1198,25 @@ public void run() { } public void notifyCheckpointComplete(final long checkpointID) { - AbstractInvokable invokable = this.invokable; + final AbstractInvokable invokable = this.invokable; if (executionState == ExecutionState.RUNNING && invokable != null) { - if (invokable instanceof StatefulTask) { - - // build a local closure - final StatefulTask statefulTask = (StatefulTask) invokable; - final String taskName = taskNameWithSubtask; - Runnable runnable = new Runnable() { - @Override - public void run() { - try { - statefulTask.notifyCheckpointComplete(checkpointID); - } - catch (Throwable t) { - if (getExecutionState() == ExecutionState.RUNNING) { - // fail task if checkpoint confirmation failed. - failExternally(new RuntimeException( - "Error while confirming checkpoint", - t)); - } + Runnable runnable = new Runnable() { + @Override + public void run() { + try { + invokable.notifyCheckpointComplete(checkpointID); + } + catch (Throwable t) { + if (getExecutionState() == ExecutionState.RUNNING) { + // fail task if checkpoint confirmation failed. + failExternally(new RuntimeException("Error while confirming checkpoint", t)); } } - }; - executeAsyncCallRunnable(runnable, "Checkpoint Confirmation for " + taskName); - } - else { - LOG.error("Task received a checkpoint commit notification, but is not a checkpoint committing task - {}.", - taskNameWithSubtask); - } + } + }; + executeAsyncCallRunnable(runnable, "Checkpoint Confirmation for " + taskNameWithSubtask); } else { LOG.debug("Ignoring checkpoint commit notification for non-running task {}.", taskNameWithSubtask); @@ -1344,7 +1285,7 @@ void onPartitionStateUpdate( /** * Utility method to dispatch an asynchronous call on the invokable. - * + * * @param runnable The async call runnable. * @param callName The name of the call, for logging purposes. */ @@ -1412,6 +1353,88 @@ public String toString() { return String.format("%s (%s) [%s]", taskNameWithSubtask, executionId, executionState); } + /** + * Instantiates the given task invokable class, passing the given environment (and possibly + * the initial task state) to the task's constructor. + * + *

The method will first try to instantiate the task via a constructor accepting both + * the Environment and the TaskStateSnapshot. If no such constructor exists, and there is + * no initial state, the method will fall back to the stateless convenience constructor that + * accepts only the Environment. + * + * @param classLoader The classloader to load the class through. + * @param className The name of the class to load. + * @param environment The task environment. + * @param initialState The task's initial state. Null, if the task is either stateless, or + * initialized with empty state. + * + * @return The instantiated invokable task object. + * + * @throws Throwable Forwards all exceptions that happen during initialization of the task. + * Also throws an exception if the task class misses the necessary constructor. + */ + private static AbstractInvokable loadAndInstantiateInvokable( + ClassLoader classLoader, + String className, + Environment environment, + @Nullable TaskStateSnapshot initialState) throws Throwable { + + final Class invokableClass; + try { + invokableClass = Class.forName(className, true, classLoader) + .asSubclass(AbstractInvokable.class); + } + catch (Throwable t) { + throw new Exception("Could not load the task's invokable class.", t); + } + + Constructor statefulCtor = null; + Constructor statelessCtor = null; + + // try to find and call the constructor that accepts state + try { + //noinspection JavaReflectionMemberAccess + statefulCtor = invokableClass.getConstructor(Environment.class, TaskStateSnapshot.class); + } + catch (NoSuchMethodException e) { + if (initialState == null) { + // we allow also the constructor that takes no state, as a convenience for stateless + // tasks so that they are not forced to carry the stateful constructor + try { + statelessCtor = invokableClass.getConstructor(Environment.class); + } + catch (NoSuchMethodException ee) { + throw new FlinkException("Task misses proper constructor", ee); + } + } + else { + throw new FlinkException("Task has state to restore, but misses the stateful constructor", e); + } + } + + // instantiate the class + try { + if (statefulCtor != null) { + return statefulCtor.newInstance(environment, initialState); + } + else { + //noinspection ConstantConditions --> cannot happen + return statelessCtor.newInstance(environment); + } + } + catch (InvocationTargetException e) { + // directly forward exceptions from the eager initialization + throw e.getTargetException(); + } + catch (Exception e) { + throw new FlinkException("Could not instantiate the task's invokable class.", e); + } + } + + // ------------------------------------------------------------------------ + // TaskCanceler + // ------------------------------------------------------------------------ + /** * This runner calls cancel() on the invokable and periodically interrupts the * thread until it has terminated. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java index 482290ac83ec5..7d6c7b53b59dd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java @@ -20,7 +20,9 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -191,24 +193,20 @@ public void testCoordinatorShutsDownOnSuccess() { } public static class BlockingInvokable extends AbstractInvokable { - private static boolean blocking = true; - private static final Object lock = new Object(); + + private static final OneShotLatch LATCH = new OneShotLatch(); + + public BlockingInvokable(Environment environment) { + super(environment); + } @Override public void invoke() throws Exception { - while (blocking) { - synchronized (lock) { - lock.wait(); - } - } + LATCH.await(); } public static void unblock() { - blocking = false; - - synchronized (lock) { - lock.notifyAll(); - } + LATCH.trigger(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java index 301d206e86562..ff15d6743a382 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -142,6 +143,10 @@ public static class BlockingTask extends AbstractInvokable { private volatile static int HasBlockedExecution = 0; private static Object waitLock = new Object(); + public BlockingTask(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { if (BlockExecution > 0) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index f57726c199f69..9481d57529c9f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; @@ -110,6 +111,10 @@ public void testPartialConsumePipelinedResultReceiver() throws Exception { */ public static class SlowBufferSender extends AbstractInvokable { + public SlowBufferSender(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { final ResultPartitionWriter writer = getEnvironment().getWriter(0); @@ -128,6 +133,10 @@ public void invoke() throws Exception { */ public static class SingleBufferReceiver extends AbstractInvokable { + public SingleBufferReceiver(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { InputGate gate = getEnvironment().getInputGate(0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java index 12bb95ecff096..79f7342b128e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.Path; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.blob.BlobServer; @@ -38,6 +39,7 @@ import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; @@ -56,7 +58,6 @@ import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -97,6 +98,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -120,6 +122,8 @@ import scala.concurrent.duration.FiniteDuration; import scala.runtime.BoxedUnit; +import javax.annotation.Nullable; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -486,28 +490,23 @@ public CheckpointIDCounter createCheckpointIDCounter(JobID jobId) throws Excepti public static class BlockingInvokable extends AbstractInvokable { - private static boolean blocking = true; - private static Object lock = new Object(); + private static final OneShotLatch LATCH = new OneShotLatch(); + + public BlockingInvokable(Environment environment, @Nullable TaskStateSnapshot initialState) { + super(environment); + } @Override public void invoke() throws Exception { - while (blocking) { - synchronized (lock) { - lock.wait(); - } - } + LATCH.await(); } public static void unblock() { - blocking = false; - - synchronized (lock) { - lock.notifyAll(); - } + LATCH.trigger(); } } - public static class BlockingStatefulInvokable extends BlockingInvokable implements StatefulTask { + public static class BlockingStatefulInvokable extends BlockingInvokable { private static final int NUM_CHECKPOINTS_TO_COMPLETE = 5; @@ -517,15 +516,18 @@ public static class BlockingStatefulInvokable extends BlockingInvokable implemen private int completedCheckpoints = 0; - @Override - public void setInitialState( - TaskStateSnapshot taskStateHandles) throws Exception { + public BlockingStatefulInvokable(Environment environment, @Nullable TaskStateSnapshot initialState) { + super(environment, initialState); + int subtaskIndex = getIndexInSubtaskGroup(); - if (subtaskIndex < recoveredStates.length) { - OperatorStateHandle operatorStateHandle = extractSingletonOperatorState(taskStateHandles); + if (initialState != null && subtaskIndex < recoveredStates.length) { + OperatorStateHandle operatorStateHandle = extractSingletonOperatorState(initialState); try (FSDataInputStream in = operatorStateHandle.openInputStream()) { recoveredStates[subtaskIndex] = InstantiationUtil.deserializeObject(in, getUserCodeClassLoader()); } + catch (IOException | ClassNotFoundException e) { + throw new RuntimeException(e.getMessage(), e); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java index 5bc207a0ec415..ecf2ae308e394 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful; import org.apache.flink.runtime.clusterframework.messages.TriggerRegistrationAtJobManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionGraph; @@ -61,7 +62,6 @@ import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.jobmanager.JobManagerHARecoveryTest.BlockingStatefulInvokable; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; @@ -1525,19 +1525,24 @@ public void testResourceManagerConnection() throws TimeoutException, Interrupted /** * A blocking stateful source task that declines savepoints. */ - public static class FailOnSavepointSourceTask extends AbstractInvokable implements StatefulTask { + public static class FailOnSavepointSourceTask extends AbstractInvokable { private static final CountDownLatch CHECKPOINT_AFTER_SAVEPOINT_LATCH = new CountDownLatch(1); private boolean receivedSavepoint; - @Override - public void invoke() throws Exception { - new CountDownLatch(1).await(); + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public FailOnSavepointSourceTask(Environment environment, TaskStateSnapshot initialState) { + super(environment); } @Override - public void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception { + public void invoke() throws Exception { + new CountDownLatch(1).await(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java index 49b11b587ff07..4ce83ccdaedd2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java @@ -19,18 +19,19 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.reader.RecordReader; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.types.IntValue; - import org.apache.flink.util.TestLogger; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -40,9 +41,9 @@ public class SlotCountExceedingParallelismTest extends TestLogger { // Test configuration - private final static int NUMBER_OF_TMS = 2; - private final static int NUMBER_OF_SLOTS_PER_TM = 2; - private final static int PARALLELISM = NUMBER_OF_TMS * NUMBER_OF_SLOTS_PER_TM; + private static final int NUMBER_OF_TMS = 2; + private static final int NUMBER_OF_SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUMBER_OF_TMS * NUMBER_OF_SLOTS_PER_TM; private static TestingCluster flink; @@ -120,7 +121,11 @@ private JobGraph createTestJobGraph( */ public static class RoundRobinSubtaskIndexSender extends AbstractInvokable { - public final static String CONFIG_KEY = "number-of-times-to-send"; + public static final String CONFIG_KEY = "number-of-times-to-send"; + + public RoundRobinSubtaskIndexSender(Environment environment) { + super(environment); + } @Override public void invoke() throws Exception { @@ -147,7 +152,11 @@ public void invoke() throws Exception { */ public static class SubtaskIndexReceiver extends AbstractInvokable { - public final static String CONFIG_KEY = "number-of-indexes-to-receive"; + public static final String CONFIG_KEY = "number-of-indexes-to-receive"; + + public SubtaskIndexReceiver(Environment environment) { + super(environment); + } @Override public void invoke() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java index d861455c53118..902c92564d062 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.jobmanager.scheduler; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -42,9 +43,9 @@ public class ScheduleOrUpdateConsumersTest extends TestLogger { - private final static int NUMBER_OF_TMS = 2; - private final static int NUMBER_OF_SLOTS_PER_TM = 2; - private final static int PARALLELISM = NUMBER_OF_TMS * NUMBER_OF_SLOTS_PER_TM; + private static final int NUMBER_OF_TMS = 2; + private static final int NUMBER_OF_SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUMBER_OF_TMS * NUMBER_OF_SLOTS_PER_TM; private static TestingCluster flink; @@ -124,7 +125,11 @@ public void testMixedPipelinedAndBlockingResults() throws Exception { public static class BinaryRoundRobinSubtaskIndexSender extends AbstractInvokable { - public final static String CONFIG_KEY = "number-of-times-to-send"; + public static final String CONFIG_KEY = "number-of-times-to-send"; + + public BinaryRoundRobinSubtaskIndexSender(Environment environment) { + super(environment); + } @Override public void invoke() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java index 1fd004f7cd1c9..c92b2f0676ef0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSinkTaskTest.java @@ -77,9 +77,9 @@ public void testDataSinkTask() { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0); - DataSinkTask testTask = new DataSinkTask<>(); + DataSinkTask testTask = new DataSinkTask<>(this.mockEnv); - super.registerFileOutputTask(testTask, MockOutputFormat.class, new File(tempTestPath).toURI().toString()); + super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString()); testTask.invoke(); @@ -139,9 +139,9 @@ public void testUnionDataSinkTask() { readers[2] = super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt * 2, 0, false), 0, false); readers[3] = super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt * 3, 0, false), 0, false); - DataSinkTask testTask = new DataSinkTask<>(); + DataSinkTask testTask = new DataSinkTask<>(this.mockEnv); - super.registerFileOutputTask(testTask, MockOutputFormat.class, new File(tempTestPath).toURI().toString()); + super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString()); try { // For the union reader to work, we need to start notifications *after* the union reader @@ -215,7 +215,7 @@ public void testSortingDataSinkTask() { super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0); - DataSinkTask testTask = new DataSinkTask<>(); + DataSinkTask testTask = new DataSinkTask<>(this.mockEnv); // set sorting super.getTaskConfig().setInputLocalStrategy(0, LocalStrategy.SORT); @@ -225,7 +225,7 @@ public void testSortingDataSinkTask() { super.getTaskConfig().setFilehandlesInput(0, 8); super.getTaskConfig().setSpillingThresholdInput(0, 0.8f); - super.registerFileOutputTask(testTask, MockOutputFormat.class, new File(tempTestPath).toURI().toString()); + super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString()); try { testTask.invoke(); @@ -293,11 +293,11 @@ public void testFailingDataSinkTask() { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0); - DataSinkTask testTask = new DataSinkTask<>(); + DataSinkTask testTask = new DataSinkTask<>(this.mockEnv); Configuration stubParams = new Configuration(); super.getTaskConfig().setStubParameters(stubParams); - super.registerFileOutputTask(testTask, MockFailingOutputFormat.class, new File(tempTestPath).toURI().toString()); + super.registerFileOutputTask(MockFailingOutputFormat.class, new File(tempTestPath).toURI().toString()); boolean stubFailed = false; @@ -325,7 +325,7 @@ public void testFailingSortingDataSinkTask() { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0); - DataSinkTask testTask = new DataSinkTask<>(); + DataSinkTask testTask = new DataSinkTask<>(this.mockEnv); Configuration stubParams = new Configuration(); super.getTaskConfig().setStubParameters(stubParams); @@ -337,7 +337,7 @@ public void testFailingSortingDataSinkTask() { super.getTaskConfig().setFilehandlesInput(0, 8); super.getTaskConfig().setSpillingThresholdInput(0, 0.8f); - super.registerFileOutputTask(testTask, MockFailingOutputFormat.class, new File(tempTestPath).toURI().toString()); + super.registerFileOutputTask(MockFailingOutputFormat.class, new File(tempTestPath).toURI().toString()); boolean stubFailed = false; @@ -359,11 +359,11 @@ public void testCancelDataSinkTask() throws Exception { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addInput(new InfiniteInputIterator(), 0); - final DataSinkTask testTask = new DataSinkTask<>(); + final DataSinkTask testTask = new DataSinkTask<>(this.mockEnv); Configuration stubParams = new Configuration(); super.getTaskConfig().setStubParameters(stubParams); - super.registerFileOutputTask(testTask, MockOutputFormat.class, new File(tempTestPath).toURI().toString()); + super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString()); Thread taskRunner = new Thread() { @Override @@ -407,7 +407,7 @@ public void testCancelSortingDataSinkTask() { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addInput(new InfiniteInputIterator(), 0); - final DataSinkTask testTask = new DataSinkTask<>(); + final DataSinkTask testTask = new DataSinkTask<>(this.mockEnv); Configuration stubParams = new Configuration(); super.getTaskConfig().setStubParameters(stubParams); @@ -419,7 +419,7 @@ public void testCancelSortingDataSinkTask() { super.getTaskConfig().setFilehandlesInput(0, 8); super.getTaskConfig().setSpillingThresholdInput(0, 0.8f); - super.registerFileOutputTask(testTask, MockOutputFormat.class, new File(tempTestPath).toURI().toString()); + super.registerFileOutputTask(MockOutputFormat.class, new File(tempTestPath).toURI().toString()); Thread taskRunner = new Thread() { @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java index 82f3d1d88c291..9a3c956e7d38b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/DataSourceTaskTest.java @@ -78,8 +78,8 @@ public void testDataSourceTask() { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addOutput(this.outList); - DataSourceTask testTask = new DataSourceTask<>(); - + DataSourceTask testTask = new DataSourceTask<>(this.mockEnv); + super.registerFileInputTask(testTask, MockInputFormat.class, new File(tempTestPath).toURI().toString(), "\n"); try { @@ -144,7 +144,7 @@ public void testFailingDataSourceTask() { super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE); super.addOutput(this.outList); - DataSourceTask testTask = new DataSourceTask<>(); + DataSourceTask testTask = new DataSourceTask<>(this.mockEnv); super.registerFileInputTask(testTask, MockFailingInputFormat.class, new File(tempTestPath).toURI().toString(), "\n"); @@ -178,7 +178,7 @@ public void testCancelDataSourceTask() { Assert.fail("Unable to set-up test input file"); } - final DataSourceTask testTask = new DataSourceTask<>(); + final DataSourceTask testTask = new DataSourceTask<>(this.mockEnv); super.registerFileInputTask(testTask, MockDelayingInputFormat.class, new File(tempTestPath).toURI().toString(), "\n"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/InvokableClassConstructorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/InvokableClassConstructorTest.java new file mode 100644 index 0000000000000..5880f0591994f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/InvokableClassConstructorTest.java @@ -0,0 +1,67 @@ +/* + * 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; + +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.iterative.task.IterationHeadTask; +import org.apache.flink.runtime.iterative.task.IterationIntermediateTask; +import org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask; +import org.apache.flink.runtime.iterative.task.IterationTailTask; + +import org.junit.Test; + +import static org.junit.Assert.fail; + +/** + * Tests that validate that stateless/stateful task implementations have the corresponding constructors. + */ +public class InvokableClassConstructorTest { + + private static final Class[] STATELESS_TASKS = { + IterationHeadTask.class, + IterationIntermediateTask.class, + IterationTailTask.class, + IterationSynchronizationSinkTask.class, + DataSourceTask.class, + DataSinkTask.class + }; + + // ------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------ + + @Test + public void testNoStatefulConstructor() throws Exception { + for (Class clazz: STATELESS_TASKS) { + + // check that there is a constructor for Environment only + clazz.getConstructor(Environment.class); + + try { + // check that there is NO constructor for Environment and Task State + clazz.getDeclaredConstructor(Environment.class, TaskStateSnapshot.class); + fail("Should fail with an exception"); + } + catch (NoSuchMethodException e) { + // expected + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java index 0a9efb0053f89..6f15884a1e935 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java @@ -27,7 +27,6 @@ import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.runtime.testutils.recordutils.RecordComparatorFactory; import org.apache.flink.runtime.testutils.recordutils.RecordSerializerFactory; -import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.operators.DriverStrategy; import org.apache.flink.runtime.operators.BatchTask; @@ -41,6 +40,7 @@ import org.apache.flink.types.IntValue; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; + import org.junit.Assert; import org.junit.Test; @@ -96,9 +96,8 @@ public void testMapTask() { // chained map+combine { - BatchTask, Record> testTask = - new BatchTask<>(); - registerTask(testTask, FlatMapDriver.class, MockMapStub.class); + registerTask(FlatMapDriver.class, MockMapStub.class); + BatchTask, Record> testTask = new BatchTask<>(this.mockEnv); try { testTask.invoke(); @@ -157,19 +156,16 @@ public void testFailingMapTask() { // chained map+combine { - final BatchTask, Record> testTask = - new BatchTask<>(); - - super.registerTask(testTask, FlatMapDriver.class, MockMapStub.class); - + registerTask(FlatMapDriver.class, MockMapStub.class); + final BatchTask, Record> testTask = new BatchTask<>(this.mockEnv); + boolean stubFailed = false; - try { testTask.invoke(); } catch (Exception e) { stubFailed = true; } - + Assert.assertTrue("Function exception was not forwarded.", stubFailed); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriverTest.java index 51ac665c1691f..a81e9597bfd25 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriverTest.java @@ -92,8 +92,8 @@ public void testMapTask() { // chained map+reduce { - BatchTask, Record> testTask = new BatchTask<>(); - registerTask(testTask, FlatMapDriver.class, MockMapStub.class); + registerTask(FlatMapDriver.class, MockMapStub.class); + BatchTask, Record> testTask = new BatchTask<>(mockEnv); try { testTask.invoke(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyInvokable.java index c47ab33d541f8..cb0faf005277e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyInvokable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyInvokable.java @@ -20,13 +20,25 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import javax.annotation.Nullable; + /** * An invokable that does nothing. */ public class DummyInvokable extends AbstractInvokable { + public DummyInvokable() { + super(new DummyEnvironment("test", 1, 0)); + } + + public DummyInvokable(Environment environment, @Nullable TaskStateSnapshot initialState) { + super(environment); + } + @Override public void invoke() {} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java index 8d198d89b2d2d..707b34ab4a50c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java @@ -43,7 +43,7 @@ import java.util.List; public abstract class TaskTestBase extends TestLogger { - + protected long memorySize = 0; protected MockInputSplitProvider inputSplitProvider; @@ -88,34 +88,26 @@ public Configuration getConfiguration() { return this.mockEnv.getTaskConfiguration(); } - public void registerTask(AbstractInvokable task, - @SuppressWarnings("rawtypes") Class driver, - Class stubClass) { - + public void registerTask( + @SuppressWarnings("rawtypes") Class driver, + Class stubClass) { + final TaskConfig config = new TaskConfig(this.mockEnv.getTaskConfiguration()); config.setDriver(driver); config.setStubWrapper(new UserCodeClassWrapper<>(stubClass)); - - task.setEnvironment(this.mockEnv); } - public void registerTask(AbstractInvokable task) { - task.setEnvironment(this.mockEnv); + public void registerFileOutputTask(Class> stubClass, String outPath) { + registerFileOutputTask(InstantiationUtil.instantiate(stubClass, FileOutputFormat.class), outPath); } - public void registerFileOutputTask(AbstractInvokable outTask, Class> stubClass, String outPath) { - registerFileOutputTask(outTask, InstantiationUtil.instantiate(stubClass, FileOutputFormat.class), outPath); - } - - public void registerFileOutputTask(AbstractInvokable outTask, FileOutputFormat outputFormat, String outPath) { + public void registerFileOutputTask(FileOutputFormat outputFormat, String outPath) { TaskConfig dsConfig = new TaskConfig(this.mockEnv.getTaskConfiguration()); - + outputFormat.setOutputFilePath(new Path(outPath)); outputFormat.setWriteMode(WriteMode.OVERWRITE); dsConfig.setStubWrapper(new UserCodeObjectWrapper<>(outputFormat)); - - outTask.setEnvironment(this.mockEnv); } public void registerFileInputTask(AbstractInvokable inTask, @@ -128,16 +120,14 @@ public void registerFileInputTask(AbstractInvokable inTask, catch (Throwable t) { throw new RuntimeException("Could not instantiate test input format.", t); } - + format.setFilePath(inPath); format.setDelimiter(delimiter); - + TaskConfig dsConfig = new TaskConfig(this.mockEnv.getTaskConfiguration()); dsConfig.setStubWrapper(new UserCodeObjectWrapper<>(format)); - - this.inputSplitProvider.addInputSplits(inPath, 5); - inTask.setEnvironment(this.mockEnv); + this.inputSplitProvider.addInputSplits(inPath, 5); } public MemoryManager getMemoryManager() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java index 6b0f251f63f57..7c9e80bc209cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.akka.AkkaJobManagerGateway; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobClient; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -312,6 +313,10 @@ private OperatorBackPressureStats triggerStatsSample( */ public static class BackPressuredTask extends AbstractInvokable { + public BackPressuredTask(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { while (true) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 08d325654a570..f882de7efe6b0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; @@ -791,6 +792,10 @@ public static class TestInvokable extends AbstractInvokable { static final CompletableFuture completableFuture = new CompletableFuture<>(); + public TestInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { completableFuture.complete(true); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index f2c60de3ee5e8..9a6f40587fe49 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -48,7 +49,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.jobgraph.tasks.StoppableTask; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; @@ -180,8 +180,8 @@ public void testThrowExceptionIfStopInvokedWithNotStoppableTask() throws Excepti } /** - * Asserts that {@link StatefulTask#triggerCheckpoint(CheckpointMetaData, CheckpointOptions)}, - * {@link StatefulTask#notifyCheckpointComplete(long)}, and {@link StoppableTask#stop()} are + * Asserts that {@link AbstractInvokable#triggerCheckpoint(CheckpointMetaData, CheckpointOptions)}, + * {@link AbstractInvokable#notifyCheckpointComplete(long)}, and {@link StoppableTask#stop()} are * invoked by a thread whose context class loader is set to the user code class loader. */ @Test @@ -213,7 +213,7 @@ private Task createTask(Class invokableClass) throw LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(new TestUserCodeClassLoader()); - + ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); ResultPartitionConsumableNotifier consumableNotifier = mock(ResultPartitionConsumableNotifier.class); PartitionProducerStateChecker partitionProducerStateChecker = mock(PartitionProducerStateChecker.class); @@ -273,16 +273,20 @@ private Task createTask(Class invokableClass) throw executor); } - public static class CheckpointsInOrderInvokable extends AbstractInvokable implements StatefulTask { + public static class CheckpointsInOrderInvokable extends AbstractInvokable { private volatile long lastCheckpointId = 0; - + private volatile Exception error; - + + public CheckpointsInOrderInvokable(Environment environment, TaskStateSnapshot initialState) { + super(environment); + } + @Override public void invoke() throws Exception { awaitLatch.trigger(); - + // wait forever (until canceled) synchronized (this) { while (error == null && lastCheckpointId < numCalls) { @@ -300,9 +304,6 @@ public void invoke() throws Exception { } } - @Override - public void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception {} - @Override public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { lastCheckpointId++; @@ -351,6 +352,10 @@ public void notifyCheckpointComplete(long checkpointId) { */ public static class ContextClassLoaderInterceptingInvokable extends CheckpointsInOrderInvokable implements StoppableTask { + public ContextClassLoaderInterceptingInvokable(Environment environment, TaskStateSnapshot initialState) { + super(environment, initialState); + } + @Override public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { classLoaders.add(Thread.currentThread().getContextClassLoader()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java index 1d09ce6ba9162..8cc936f01f346 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; @@ -180,6 +181,10 @@ public void testCancelAsyncProducerAndConsumer() throws Exception { */ public static class AsyncProducer extends AbstractInvokable { + public AsyncProducer(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { Thread producer = new ProducerThread(getEnvironment().getWriter(0)); @@ -233,6 +238,10 @@ public void run() { */ public static class AsyncConsumer extends AbstractInvokable { + public AsyncConsumer(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { Thread consumer = new ConsumerThread(getEnvironment().getInputGate(0)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 31e9e223b6900..4a291f279707d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionLocation; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; @@ -2047,12 +2048,20 @@ public SimplePartitionStateLookupJobManager create() throws Exception { public static final class TestInvokableCorrect extends AbstractInvokable { + public TestInvokableCorrect(Environment environment) { + super(environment); + } + @Override public void invoke() {} } public static class TestInvokableBlockingCancelable extends AbstractInvokable { + public TestInvokableBlockingCancelable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { final Object o = new Object(); @@ -2071,6 +2080,10 @@ public static final class TestInvokableRecordCancel extends AbstractInvokable { private static final Object lock = new Object(); private static CompletableFuture gotCanceledFuture = new CompletableFuture<>(); + public TestInvokableRecordCancel(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { final Object o = new Object(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 4fa36bb23d571..ade6a662d4dc5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.CancelTaskException; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -1159,6 +1160,10 @@ private void validateCancelingAndCanceledListenerMessage(Task task) { public static final class TestInvokableCorrect extends AbstractInvokable { + public TestInvokableCorrect(Environment environment) { + super(environment); + } + @Override public void invoke() {} @@ -1169,7 +1174,11 @@ public void cancel() throws Exception { } public static final class InvokableWithExceptionInInvoke extends AbstractInvokable { - + + public InvokableWithExceptionInInvoke(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { throw new Exception("test"); @@ -1178,6 +1187,10 @@ public void invoke() throws Exception { public static final class InvokableWithExceptionOnTrigger extends AbstractInvokable { + public InvokableWithExceptionOnTrigger(Environment environment) { + super(environment); + } + @Override public void invoke() { awaitLatch.trigger(); @@ -1198,10 +1211,19 @@ public void invoke() { } } - public static abstract class InvokableNonInstantiable extends AbstractInvokable {} + public static abstract class InvokableNonInstantiable extends AbstractInvokable { + + public InvokableNonInstantiable(Environment environment) { + super(environment); + } + } public static final class InvokableBlockingInInvoke extends AbstractInvokable { + public InvokableBlockingInInvoke(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { awaitLatch.trigger(); @@ -1215,6 +1237,10 @@ public void invoke() throws Exception { public static final class InvokableWithCancelTaskExceptionInInvoke extends AbstractInvokable { + public InvokableWithCancelTaskExceptionInInvoke(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { awaitLatch.trigger(); @@ -1232,6 +1258,10 @@ public static final class InvokableInterruptableSharedLockInInvokeAndCancel exte private final Object lock = new Object(); + public InvokableInterruptableSharedLockInInvokeAndCancel(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { synchronized (lock) { @@ -1250,6 +1280,10 @@ public void cancel() throws Exception { public static final class InvokableBlockingInCancel extends AbstractInvokable { + public InvokableBlockingInCancel(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { awaitLatch.trigger(); @@ -1277,6 +1311,10 @@ public void cancel() throws Exception { public static final class InvokableUninterruptibleBlockingInvoke extends AbstractInvokable { + public InvokableUninterruptibleBlockingInvoke(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { while (!cancelLatch.isTriggered()) { @@ -1297,6 +1335,10 @@ public void cancel() throws Exception { public static final class FailingInvokableWithChainedException extends AbstractInvokable { + public FailingInvokableWithChainedException(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { throw new TestWrappedException(new IOException("test")); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java index c9adba8a0394a..56c88d26aa50b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/BlockingNoOpInvokable.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.testtasks; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; /** @@ -25,6 +26,10 @@ */ public class BlockingNoOpInvokable extends AbstractInvokable { + public BlockingNoOpInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { final Object o = new Object(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java index 37c141d6b9f02..807106811d0e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.testtasks; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; /** @@ -28,6 +29,15 @@ public class FailingBlockingInvokable extends AbstractInvokable { private static volatile boolean blocking = true; private static final Object lock = new Object(); + /** + * Create an Invokable task and set its environment. + * + * @param environment The environment assigned to this invokable. + */ + public FailingBlockingInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { while (blocking) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java index fa9949ab65044..7d50e94fea0b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/NoOpInvokable.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.testtasks; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; /** @@ -25,6 +26,10 @@ */ public class NoOpInvokable extends AbstractInvokable { + public NoOpInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java index de7d59a925065..44afd5e2143f8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/WaitingNoOpInvokable.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.testtasks; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; /** @@ -27,6 +28,10 @@ public class WaitingNoOpInvokable extends AbstractInvokable { private static final long waitingTime = 100L; + public WaitingNoOpInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { Thread.sleep(waitingTime); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/StoppableInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/StoppableInvokable.java index 10d2eff575d13..ba8f7f657351b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/StoppableInvokable.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/StoppableInvokable.java @@ -10,17 +10,24 @@ * 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.testutils; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.StoppableTask; public final class StoppableInvokable extends AbstractInvokable implements StoppableTask { + private boolean isRunning = true; + public StoppableInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { - while(isRunning) { + while (isRunning) { Thread.sleep(100); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index 086ad71b65b12..43e2b8da8626d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; @@ -213,6 +214,10 @@ public static void main(String[] args) throws Exception { public static final class OomInvokable extends AbstractInvokable { + public OomInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { throw new OutOfMemoryError(); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala index fabd66b106dca..bf2f3ba6c426e 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala @@ -18,6 +18,7 @@ package org.apache.flink.runtime.jobmanager +import org.apache.flink.runtime.execution.Environment import org.apache.flink.runtime.io.network.api.reader.RecordReader import org.apache.flink.runtime.io.network.api.writer.RecordWriter import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable @@ -26,7 +27,8 @@ import org.apache.flink.types.IntValue object Tasks { - class Sender extends AbstractInvokable{ + class Sender(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { val writer = new RecordWriter[IntValue](getEnvironment.getWriter(0)) @@ -41,7 +43,8 @@ object Tasks { } } - class Forwarder extends AbstractInvokable { + class Forwarder(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { val reader = new RecordReader[IntValue]( @@ -69,7 +72,8 @@ object Tasks { } } - class Receiver extends AbstractInvokable { + class Receiver(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { val reader = new RecordReader[IntValue]( @@ -87,7 +91,8 @@ object Tasks { } } - class FailingOnceReceiver extends Receiver { + class FailingOnceReceiver(environment: Environment) + extends Receiver(environment) { import FailingOnceReceiver.failed override def invoke(): Unit = { @@ -104,7 +109,8 @@ object Tasks { var failed = false } - class BlockingOnceReceiver extends Receiver { + class BlockingOnceReceiver(environment: Environment) + extends Receiver(environment) { import BlockingOnceReceiver.blocking override def invoke(): Unit = { @@ -124,7 +130,8 @@ object Tasks { var blocking = true } - class AgnosticReceiver extends AbstractInvokable { + class AgnosticReceiver(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { val reader= new RecordReader[IntValue]( @@ -136,7 +143,8 @@ object Tasks { } } - class AgnosticBinaryReceiver extends AbstractInvokable { + class AgnosticBinaryReceiver(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { val reader1 = new RecordReader[IntValue]( @@ -154,7 +162,8 @@ object Tasks { } } - class AgnosticTertiaryReceiver extends AbstractInvokable { + class AgnosticTertiaryReceiver(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { val env = getEnvironment @@ -180,14 +189,16 @@ object Tasks { } } - class ExceptionSender extends AbstractInvokable{ + class ExceptionSender(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { throw new Exception("Test exception") } } - class SometimesExceptionSender extends AbstractInvokable { + class SometimesExceptionSender(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { // this only works if the TaskManager runs in the same JVM as the test case @@ -204,21 +215,24 @@ object Tasks { var failingSenders = Set[Int](0) } - class ExceptionReceiver extends AbstractInvokable { + class ExceptionReceiver(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { throw new Exception("Test exception") } } - class InstantiationErrorSender extends AbstractInvokable{ + class InstantiationErrorSender(environment: Environment) + extends AbstractInvokable(environment) { throw new RuntimeException("Test exception in constructor") override def invoke(): Unit = { } } - class SometimesInstantiationErrorSender extends AbstractInvokable{ + class SometimesInstantiationErrorSender(environment: Environment) + extends AbstractInvokable(environment) { // this only works if the TaskManager runs in the same JVM as the test case if(SometimesInstantiationErrorSender.failingSenders.contains(this.getIndexInSubtaskGroup)){ @@ -235,7 +249,8 @@ object Tasks { var failingSenders = Set[Int](0) } - class BlockingReceiver extends AbstractInvokable { + class BlockingReceiver(environment: Environment) + extends AbstractInvokable(environment) { override def invoke(): Unit = { val o = new Object o.synchronized( diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java index e66aceca31d6e..ecfd7320f0bc8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java @@ -31,7 +31,7 @@ import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,7 +85,7 @@ public class BarrierBuffer implements CheckpointBarrierHandler { private BufferSpiller.SpilledBufferOrEventSequence currentBuffered; /** Handler that receives the checkpoint notifications. */ - private StatefulTask toNotifyOnCheckpoint; + private AbstractInvokable toNotifyOnCheckpoint; /** The ID of the checkpoint for which we expect barriers. */ private long currentCheckpointId = -1L; @@ -406,7 +406,7 @@ private void checkSizeLimit() throws Exception { } @Override - public void registerCheckpointEventHandler(StatefulTask toNotifyOnCheckpoint) { + public void registerCheckpointEventHandler(AbstractInvokable toNotifyOnCheckpoint) { if (this.toNotifyOnCheckpoint == null) { this.toNotifyOnCheckpoint = toNotifyOnCheckpoint; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java index 9d28a4fc11076..8178fbcf6d6e8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,7 +74,7 @@ public class BarrierTracker implements CheckpointBarrierHandler { private final ArrayDeque pendingCheckpoints; /** The listener to be notified on complete checkpoints. */ - private StatefulTask toNotifyOnCheckpoint; + private AbstractInvokable toNotifyOnCheckpoint; /** The highest checkpoint ID encountered so far. */ private long latestPendingCheckpointID = -1; @@ -84,7 +84,7 @@ public class BarrierTracker implements CheckpointBarrierHandler { public BarrierTracker(InputGate inputGate) { this.inputGate = inputGate; this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels(); - this.pendingCheckpoints = new ArrayDeque(); + this.pendingCheckpoints = new ArrayDeque<>(); } @Override @@ -109,7 +109,7 @@ else if (next.getEvent().getClass() == CancelCheckpointMarker.class) { } @Override - public void registerCheckpointEventHandler(StatefulTask toNotifyOnCheckpoint) { + public void registerCheckpointEventHandler(AbstractInvokable toNotifyOnCheckpoint) { if (this.toNotifyOnCheckpoint == null) { this.toNotifyOnCheckpoint = toNotifyOnCheckpoint; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java index 0c676e051ce31..8662a950a42b6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import java.io.IOException; @@ -53,7 +53,7 @@ public interface CheckpointBarrierHandler { * * @param task The task to notify */ - void registerCheckpointEventHandler(StatefulTask task); + void registerCheckpointEventHandler(AbstractInvokable task); /** * Cleans up all internally held resources. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 0c71a53782be5..cf8f452e2da42 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -35,7 +35,6 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.runtime.plugable.DeserializationDelegate; @@ -49,6 +48,7 @@ import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; +import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -115,7 +115,7 @@ public class StreamInputProcessor { public StreamInputProcessor( InputGate[] inputGates, TypeSerializer inputSerializer, - StatefulTask checkpointedTask, + StreamTask checkpointedTask, CheckpointingMode checkpointMode, Object lock, IOManager ioManager, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 824acad488967..b99ad5375e5f9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -35,7 +35,6 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.runtime.plugable.DeserializationDelegate; @@ -49,6 +48,7 @@ import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; +import org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,7 +133,7 @@ public StreamTwoInputProcessor( Collection inputGates2, TypeSerializer inputSerializer1, TypeSerializer inputSerializer2, - StatefulTask checkpointedTask, + TwoInputStreamTask checkpointedTask, CheckpointingMode checkpointMode, Object lock, IOManager ioManager, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index f461e31540f0b..8873b55acbd9b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -19,12 +19,17 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.io.StreamInputProcessor; +import javax.annotation.Nullable; + /** * A {@link StreamTask} for executing a {@link OneInputStreamOperator}. */ @@ -35,6 +40,35 @@ public class OneInputStreamTask extends StreamTaskThis constructor accepts a special {@link ProcessingTimeService}. By default (and if + * null is passes for the time provider) a {@link SystemProcessingTimeService DefaultTimerService} + * will be used. + * + * @param env The task environment for this task. + * @param initialState The initial state for this task (null indicates no initial state) + * @param timeProvider Optionally, a specific time provider to use. + */ + @VisibleForTesting + public OneInputStreamTask( + Environment env, + @Nullable TaskStateSnapshot initialState, + @Nullable ProcessingTimeService timeProvider) { + super(env, initialState, timeProvider); + } + @Override public void init() throws Exception { StreamConfig configuration = getConfiguration(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java index 408c843466087..2a0787cb4d28d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java @@ -21,11 +21,15 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.util.FlinkException; +import javax.annotation.Nullable; + /** * {@link StreamTask} for executing a {@link StreamSource}. * @@ -46,6 +50,10 @@ public class SourceStreamTask, OP extends S private volatile boolean externallyInducedCheckpoints; + public SourceStreamTask(Environment env, @Nullable TaskStateSnapshot initialState) { + super(env, initialState); + } + @Override protected void init() { // does not hold any resources, so no initialization needed diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StoppableSourceStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StoppableSourceStreamTask.java index d2a3f4df88efc..2cda01267c55a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StoppableSourceStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StoppableSourceStreamTask.java @@ -19,6 +19,8 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.functions.StoppableFunction; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.StoppableTask; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StoppableStreamSource; @@ -34,6 +36,10 @@ public class StoppableSourceStreamTask & St private volatile boolean stopped; + public StoppableSourceStreamTask(Environment environment, TaskStateSnapshot initialState) { + super(environment, initialState); + } + @Override protected void run() throws Exception { if (!stopped) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java index b800206a33022..9f74bf4fb844c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java @@ -19,6 +19,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.BlockingQueueBroker; import org.apache.flink.streaming.runtime.io.RecordWriterOutput; @@ -27,6 +29,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; @@ -42,6 +46,10 @@ public class StreamIterationHead extends OneInputStreamTask { private volatile boolean running = true; + public StreamIterationHead(Environment env, @Nullable TaskStateSnapshot initialState) { + super(env, initialState); + } + // ------------------------------------------------------------------------ @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java index d2ec08fdc85c4..57d88031bc91d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; @@ -42,6 +44,10 @@ public class StreamIterationTail extends OneInputStreamTask { private static final Logger LOG = LoggerFactory.getLogger(StreamIterationTail.class); + public StreamIterationTail(Environment environment, TaskStateSnapshot initialState) { + super(environment, initialState); + } + @Override public void init() throws Exception { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index a3b16e6dab973..a1bd3551b95c7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -36,7 +35,6 @@ import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.CheckpointStreamFactory; @@ -62,6 +60,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.Closeable; import java.io.IOException; import java.util.Collection; @@ -116,7 +116,7 @@ @Internal public abstract class StreamTask> extends AbstractInvokable - implements StatefulTask, AsyncExceptionHandler { + implements AsyncExceptionHandler { /** The thread group that holds all trigger timer threads. */ public static final ThreadGroup TRIGGER_THREAD_GROUP = new ThreadGroup("Triggers"); @@ -157,6 +157,7 @@ public abstract class StreamTask> /** The map of user-defined accumulators of this task. */ private Map> accumulatorMap; + @Nullable private TaskStateSnapshot taskStateSnapshot; /** The currently active background materialization threads. */ @@ -180,6 +181,43 @@ public abstract class StreamTask> /** Wrapper for synchronousCheckpointExceptionHandler to deal with rethrown exceptions. Used in the async part. */ private AsyncCheckpointExceptionHandler asynchronousCheckpointExceptionHandler; + // ------------------------------------------------------------------------ + + /** + * Constructor for initialization, possibly with initial state (recovery / savepoint / etc). + * + * @param env The task environment for this task. + * @param initialState The initial state for this task (null indicates no initial state) + */ + protected StreamTask(Environment env, @Nullable TaskStateSnapshot initialState) { + this(env, initialState, null); + } + + /** + * Constructor for initialization, possibly with initial state (recovery / savepoint / etc). + * + *

This constructor accepts a special {@link ProcessingTimeService}. By default (and if + * null is passes for the time provider) a {@link SystemProcessingTimeService DefaultTimerService} + * will be used. + * + * @param env The task environment for this task. + * @param initialState The initial state for this task (null indicates no initial state) + * @param timeProvider Optionally, a specific time provider to use. + */ + protected StreamTask( + Environment env, + @Nullable TaskStateSnapshot initialState, + @Nullable ProcessingTimeService timeProvider) { + + super(env); + + // assign the initial state - later we do all initialization based on the initial state here. + this.taskStateSnapshot = initialState; + + // assign a possibly injected timer service + this.timerService = timeProvider; + } + // ------------------------------------------------------------------------ // Life cycle methods for specific implementations // ------------------------------------------------------------------------ @@ -196,21 +234,6 @@ public abstract class StreamTask> // Core work methods of the Stream Task // ------------------------------------------------------------------------ - /** - * Allows the user to specify his own {@link ProcessingTimeService TimerServiceProvider}. - * By default a {@link SystemProcessingTimeService DefaultTimerService} is going to be provided. - * Changing it can be useful for testing processing time functionality, such as - * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner WindowAssigners} - * and {@link org.apache.flink.streaming.api.windowing.triggers.Trigger Triggers}. - * */ - @VisibleForTesting - public void setProcessingTimeService(ProcessingTimeService timeProvider) { - if (timeProvider == null) { - throw new RuntimeException("The timeProvider cannot be set to null."); - } - timerService = timeProvider; - } - @Override public final void invoke() throws Exception { @@ -536,11 +559,6 @@ RecordWriterOutput[] getStreamOutputs() { // Checkpoint and Restore // ------------------------------------------------------------------------ - @Override - public void setInitialState(TaskStateSnapshot taskStateHandles) { - this.taskStateSnapshot = taskStateHandles; - } - @Override public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) throws Exception { try { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index 68722dbe8d3da..2835e4ecb641b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -19,12 +19,16 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.List; @@ -38,6 +42,16 @@ public class TwoInputStreamTask extends StreamTask task = new OneInputStreamTask<>(); - final OneInputStreamTaskTestHarness testHarness = - new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 1, 1, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); testHarness.taskConfig = chainedVertex.getConfiguration(); @@ -487,9 +488,11 @@ public Integer map(Integer value) throws Exception { @Test public void testStateSnapshotAndRestore() throws Exception { - final OneInputStreamTask task = new OneInputStreamTask<>(); - final OneInputStreamTaskTestHarness testHarness = - new OneInputStreamTaskTestHarness<>(task, 1, 1, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 1, 1, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); AsyncWaitOperator operator = new AsyncWaitOperator<>( @@ -514,6 +517,8 @@ public void testStateSnapshotAndRestore() throws Exception { testHarness.invoke(env); testHarness.waitForTaskRunning(); + final OneInputStreamTask task = testHarness.getTask(); + final long initialTime = 0L; testHarness.processElement(new StreamRecord<>(1, initialTime + 1)); @@ -540,12 +545,12 @@ public void testStateSnapshotAndRestore() throws Exception { testHarness.waitForTaskCompletion(); // set the operator state from previous attempt into the restored one - final OneInputStreamTask restoredTask = new OneInputStreamTask<>(); TaskStateSnapshot subtaskStates = env.getCheckpointStateHandles(); - restoredTask.setInitialState(subtaskStates); final OneInputStreamTaskTestHarness restoredTaskHarness = - new OneInputStreamTaskTestHarness<>(restoredTask, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); restoredTaskHarness.setupOutputForSingletonOperatorChain(); AsyncWaitOperator restoredOperator = new AsyncWaitOperator<>( @@ -557,9 +562,11 @@ public void testStateSnapshotAndRestore() throws Exception { restoredTaskHarness.getStreamConfig().setStreamOperator(restoredOperator); restoredTaskHarness.getStreamConfig().setOperatorID(operatorID); - restoredTaskHarness.invoke(); + restoredTaskHarness.invoke(subtaskStates); restoredTaskHarness.waitForTaskRunning(); + final OneInputStreamTask restoredTask = restoredTaskHarness.getTask(); + restoredTaskHarness.processElement(new StreamRecord<>(5, initialTime + 5)); restoredTaskHarness.processElement(new StreamRecord<>(6, initialTime + 6)); restoredTaskHarness.processElement(new StreamRecord<>(7, initialTime + 7)); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java index 4eda5ad3028c6..9c4a020465ff3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java @@ -33,8 +33,8 @@ public class StreamIterationHeadTest { @Test public void testIterationHeadWatermarkEmission() throws Exception { - StreamIterationHead head = new StreamIterationHead<>(); - StreamTaskTestHarness harness = new StreamTaskTestHarness<>(head, + StreamTaskTestHarness harness = new StreamTaskTestHarness<>( + StreamIterationHead::new, BasicTypeInfo.INT_TYPE_INFO); harness.setupOutputForSingletonOperatorChain(); harness.getStreamConfig().setIterationId("1"); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java index 40303cee0d513..3899fe1cb0e27 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferAlignmentLimitTest.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -116,7 +116,7 @@ public void testBreakCheckpointAtAlignmentLimit() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager, 1000); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); // validating the sequence of buffers @@ -210,7 +210,7 @@ public void testAlignmentLimitWithQueuedAlignments() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager, 500); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); // validating the sequence of buffers diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index baac79cc3a762..57ecf3a1500f3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -23,7 +23,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineOnCancellationBarrierException; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineSubsumedException; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -34,7 +33,8 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -550,7 +550,7 @@ public void testMultiChannelSkippingCheckpoints() { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); long startTs; @@ -1005,7 +1005,7 @@ public void testSingleChannelAbortCheckpoint() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); check(sequence[0], buffer.getNextNonBlocked()); @@ -1069,7 +1069,7 @@ public void testMultiChannelAbortCheckpoint() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); long startTs; @@ -1163,7 +1163,7 @@ public void testAbortViaQueuedBarriers() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); long startTs; @@ -1254,7 +1254,7 @@ public void testAbortWhileHavingQueuedBarriers() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); long startTs; @@ -1339,7 +1339,7 @@ public void testIgnoreCancelBarrierIfCheckpointSubsumed() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierBuffer buffer = new BarrierBuffer(gate, ioManager); - StatefulTask toNotify = mock(StatefulTask.class); + AbstractInvokable toNotify = mock(AbstractInvokable.class); buffer.registerCheckpointEventHandler(toNotify); long startTs; @@ -1466,11 +1466,15 @@ private static void validateAlignmentBuffered(long actualBytesBuffered, BufferOr // Testing Mocks // ------------------------------------------------------------------------ - private static class ValidatingCheckpointHandler implements StatefulTask { + private static class ValidatingCheckpointHandler extends AbstractInvokable { private long nextExpectedCheckpointId = -1L; private long lastReportedBytesBufferedInAlignment = -1; + public ValidatingCheckpointHandler() { + super(new DummyEnvironment("test", 1, 0)); + } + public void setNextExpectedCheckpointId(long nextExpectedCheckpointId) { this.nextExpectedCheckpointId = nextExpectedCheckpointId; } @@ -1484,8 +1488,8 @@ long getLastReportedBytesBufferedInAlignment() { } @Override - public void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception { - throw new UnsupportedOperationException("should never be called"); + public void invoke() { + throw new UnsupportedOperationException(); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java index 9251ab5ab2f30..535fdb97487f5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java @@ -22,13 +22,13 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; -import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.junit.Test; @@ -452,7 +452,7 @@ public void testInterleavedCancellationBarriers() throws Exception { MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence)); BarrierTracker tracker = new BarrierTracker(gate); - StatefulTask statefulTask = mock(StatefulTask.class); + AbstractInvokable statefulTask = mock(AbstractInvokable.class); tracker.registerCheckpointEventHandler(statefulTask); @@ -487,18 +487,19 @@ private static BufferOrEvent createBuffer(int channel) { // Testing Mocks // ------------------------------------------------------------------------ - private static class CheckpointSequenceValidator implements StatefulTask { + private static class CheckpointSequenceValidator extends AbstractInvokable { private final long[] checkpointIDs; private int i = 0; private CheckpointSequenceValidator(long... checkpointIDs) { + super(new DummyEnvironment("test", 1, 0)); this.checkpointIDs = checkpointIDs; } @Override - public void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception { + public void invoke() { throw new UnsupportedOperationException("should never be called"); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java index 65e59f8ac756c..70a669ccb4a51 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java @@ -44,10 +44,12 @@ public class StreamTaskTimerTest { @Test public void testOpenCloseAndTimestamps() throws Exception { - final OneInputStreamTask mapTask = new OneInputStreamTask<>(); final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( - mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + OneInputStreamTask::new, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -59,6 +61,8 @@ public void testOpenCloseAndTimestamps() throws Exception { testHarness.invoke(); testHarness.waitForTaskRunning(); + final OneInputStreamTask mapTask = testHarness.getTask(); + // first one spawns thread mapTask.getProcessingTimeService().registerTimer(System.currentTimeMillis(), new ProcessingTimeCallback() { @Override @@ -84,8 +88,11 @@ public void onProcessingTime(long timestamp) { @Test public void checkScheduledTimestampe() { try { - final OneInputStreamTask mapTask = new OneInputStreamTask<>(); - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>(mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -95,6 +102,8 @@ public void checkScheduledTimestampe() { testHarness.invoke(); testHarness.waitForTaskRunning(); + final OneInputStreamTask mapTask = testHarness.getTask(); + final AtomicReference errorRef = new AtomicReference<>(); final long t1 = System.currentTimeMillis(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java index d621b0bb12adb..008c84840561f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java @@ -41,13 +41,13 @@ public class TestProcessingTimeServiceTest { @Test public void testCustomTimeServiceProvider() throws Throwable { - TestProcessingTimeService tp = new TestProcessingTimeService(); - - final OneInputStreamTask mapTask = new OneInputStreamTask<>(); - mapTask.setProcessingTimeService(tp); + final TestProcessingTimeService tp = new TestProcessingTimeService(); final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( - mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + (env, state) -> new OneInputStreamTask<>(env, state, tp), + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -58,6 +58,8 @@ public void testCustomTimeServiceProvider() throws Throwable { testHarness.invoke(); + final OneInputStreamTask mapTask = testHarness.getTask(); + assertEquals(Long.MIN_VALUE, testHarness.getProcessingTimeService().getCurrentProcessingTime()); tp.setCurrentTime(11); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerConfigurationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerConfigurationTest.java index 69f693505925d..e47dd0be6a48b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerConfigurationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerConfigurationTest.java @@ -57,8 +57,10 @@ public void testFailIsDefaultConfig() { private void testConfigForwarding(boolean failOnException) throws Exception { final boolean expectedHandlerFlag = failOnException; - DummyEnvironment environment = new DummyEnvironment("test", 1, 0); + + final DummyEnvironment environment = new DummyEnvironment("test", 1, 0); environment.getExecutionConfig().setFailTaskOnCheckpointError(expectedHandlerFlag); + final CheckpointExceptionHandlerFactory inspectingFactory = new CheckpointExceptionHandlerFactory() { @Override @@ -71,26 +73,18 @@ public CheckpointExceptionHandler createCheckpointExceptionHandler( } }; - StreamTask streamTask = new StreamTask() { + StreamTask streamTask = new StreamTask(environment, null) { @Override - protected void init() throws Exception { - - } + protected void init() throws Exception {} @Override - protected void run() throws Exception { - - } + protected void run() throws Exception {} @Override - protected void cleanup() throws Exception { - - } + protected void cleanup() throws Exception {} @Override - protected void cancelTask() throws Exception { - - } + protected void cancelTask() throws Exception {} @Override protected CheckpointExceptionHandlerFactory createCheckpointExceptionHandlerFactory() { @@ -98,7 +92,6 @@ protected CheckpointExceptionHandlerFactory createCheckpointExceptionHandlerFact } }; - streamTask.setEnvironment(environment); streamTask.invoke(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 1fdd922b58d69..c2c255321ec60 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -89,8 +89,9 @@ public class OneInputStreamTaskTest extends TestLogger { */ @Test public void testOpenCloseAndTimestamps() throws Exception { - final OneInputStreamTask mapTask = new OneInputStreamTask(); - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -128,10 +129,11 @@ public void testOpenCloseAndTimestamps() throws Exception { @Test @SuppressWarnings("unchecked") public void testWatermarkAndStreamStatusForwarding() throws Exception { - final OneInputStreamTask mapTask = new OneInputStreamTask(); + final OneInputStreamTaskTestHarness testHarness = - new OneInputStreamTaskTestHarness( - mapTask, 2, 2, + new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); @@ -240,10 +242,11 @@ public void testWatermarkAndStreamStatusForwarding() throws Exception { */ @Test public void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { - final OneInputStreamTask testTask = new OneInputStreamTask<>(); + final OneInputStreamTaskTestHarness testHarness = - new OneInputStreamTaskTestHarness( - testTask, 1, 1, + new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 1, 1, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); @@ -340,8 +343,11 @@ public void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { */ @Test public void testCheckpointBarriers() throws Exception { - final OneInputStreamTask mapTask = new OneInputStreamTask(); - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 2, 2, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -400,8 +406,11 @@ public void testCheckpointBarriers() throws Exception { */ @Test public void testOvertakingCheckpointBarriers() throws Exception { - final OneInputStreamTask mapTask = new OneInputStreamTask(); - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 2, 2, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -471,8 +480,10 @@ public void testOvertakingCheckpointBarriers() throws Exception { @Test public void testSnapshottingAndRestoring() throws Exception { final Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); - final OneInputStreamTask streamTask = new OneInputStreamTask(); - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness(streamTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); IdentityKeySelector keySelector = new IdentityKeySelector<>(); @@ -500,6 +511,8 @@ public void testSnapshottingAndRestoring() throws Exception { testHarness.invoke(env); testHarness.waitForTaskRunning(deadline.timeLeft().toMillis()); + final OneInputStreamTask streamTask = testHarness.getTask(); + CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, checkpointTimestamp); while (!streamTask.triggerCheckpoint(checkpointMetaData, CheckpointOptions.forCheckpoint())) {} @@ -514,10 +527,10 @@ public void testSnapshottingAndRestoring() throws Exception { testHarness.endInput(); testHarness.waitForTaskCompletion(deadline.timeLeft().toMillis()); - final OneInputStreamTask restoredTask = new OneInputStreamTask(); - final OneInputStreamTaskTestHarness restoredTaskHarness = - new OneInputStreamTaskTestHarness(restoredTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + restoredTaskHarness.configureForKeyedStream(keySelector, BasicTypeInfo.STRING_TYPE_INFO); StreamConfig restoredTaskStreamConfig = restoredTaskHarness.getStreamConfig(); @@ -527,11 +540,9 @@ public void testSnapshottingAndRestoring() throws Exception { TaskStateSnapshot stateHandles = env.getCheckpointStateHandles(); Assert.assertEquals(numberChainedTasks, stateHandles.getSubtaskStateMappings().size()); - restoredTask.setInitialState(stateHandles); - TestingStreamOperator.numberRestoreCalls = 0; - restoredTaskHarness.invoke(); + restoredTaskHarness.invoke(stateHandles); restoredTaskHarness.endInput(); restoredTaskHarness.waitForTaskCompletion(deadline.timeLeft().toMillis()); @@ -544,16 +555,10 @@ public void testSnapshottingAndRestoring() throws Exception { @Test public void testQuiesceTimerServiceAfterOpClose() throws Exception { - final OneInputStreamTask task = new OneInputStreamTask<>(); - - SystemProcessingTimeService timeService = new SystemProcessingTimeService(task, task.getCheckpointLock()); - task.setProcessingTimeService(timeService); - - // verify that the timer service is running - Assert.assertTrue(((SystemProcessingTimeService) task.getProcessingTimeService()).isAlive()); - - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness( - task, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 2, 2, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -562,6 +567,13 @@ public void testQuiesceTimerServiceAfterOpClose() throws Exception { testHarness.invoke(); testHarness.waitForTaskRunning(); + + SystemProcessingTimeService timeService = (SystemProcessingTimeService) + testHarness.getTask().getProcessingTimeService(); + + // verify that the timer service is running + Assert.assertTrue(timeService.isAlive()); + testHarness.endInput(); testHarness.waitForTaskCompletion(); timeService.shutdownService(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java index 4d11c971dabcb..357d6296bf207 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java @@ -22,10 +22,13 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import java.io.IOException; +import java.util.function.BiFunction; /** @@ -57,13 +60,13 @@ public class OneInputStreamTaskTestHarness extends StreamTaskTestHarnes * of channels per input gate. */ public OneInputStreamTaskTestHarness( - OneInputStreamTask task, - int numInputGates, - int numInputChannelsPerGate, - TypeInformation inputType, - TypeInformation outputType) { + BiFunction> taskFactory, + int numInputGates, + int numInputChannelsPerGate, + TypeInformation inputType, + TypeInformation outputType) { - super(task, outputType); + super(taskFactory, outputType); this.inputType = inputType; inputSerializer = inputType.createSerializer(executionConfig); @@ -76,10 +79,11 @@ public OneInputStreamTaskTestHarness( * Creates a test harness with one input gate that has one input channel. */ public OneInputStreamTaskTestHarness( - OneInputStreamTask task, - TypeInformation inputType, - TypeInformation outputType) { - this(task, 1, 1, inputType, outputType); + BiFunction> taskFactory, + TypeInformation inputType, + TypeInformation outputType) { + + this(taskFactory, 1, 1, inputType, outputType); } @Override @@ -105,5 +109,11 @@ public void configureForKeyedStream( streamConfig.setStatePartitioner(0, keySelector); streamConfig.setStateKeySerializer(keyType.createSerializer(executionConfig)); } + + @Override + @SuppressWarnings("unchecked") + public OneInputStreamTask getTask() { + return (OneInputStreamTask) super.getTask(); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java index d54bba83a8154..b241d05daa5da 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java @@ -42,6 +42,7 @@ import org.junit.Before; import org.junit.Test; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.Optional; @@ -86,12 +87,7 @@ public void testRestore() throws Exception { new CounterOperator(), Optional.of(stateHandles)); - assertEquals( - new HashSet() {{ - add(headOperatorID); - add(tailOperatorID); - }}, - RESTORED_OPERATORS); + assertEquals(new HashSet<>(Arrays.asList(headOperatorID, tailOperatorID)), RESTORED_OPERATORS); } @Test @@ -115,11 +111,7 @@ public void testRestoreHeadWithNewId() throws Exception { new CounterOperator(), Optional.of(stateHandles)); - assertEquals( - new HashSet() {{ - add(tailOperatorID); - }}, - RESTORED_OPERATORS); + assertEquals(Collections.singleton(tailOperatorID), RESTORED_OPERATORS); } @Test @@ -144,11 +136,7 @@ public void testRestoreTailWithNewId() throws Exception { new CounterOperator(), Optional.of(stateHandles)); - assertEquals( - new HashSet() {{ - add(headOperatorID); - }}, - RESTORED_OPERATORS); + assertEquals(Collections.singleton(headOperatorID), RESTORED_OPERATORS); } @Test @@ -183,12 +171,7 @@ public void testRestoreAfterScaleUp() throws Exception { new CounterOperator(), Optional.of(stateHandles)); - assertEquals( - new HashSet() {{ - add(headOperatorID); - add(tailOperatorID); - }}, - RESTORED_OPERATORS); + assertEquals(new HashSet<>(Arrays.asList(headOperatorID, tailOperatorID)), RESTORED_OPERATORS); } @Test @@ -214,12 +197,7 @@ public void testRestoreWithoutState() throws Exception { new CounterOperator(), Optional.of(stateHandles)); - assertEquals( - new HashSet() {{ - add(headOperatorID); - add(tailOperatorID); - }}, - RESTORED_OPERATORS); + assertEquals(new HashSet<>(Arrays.asList(headOperatorID, tailOperatorID)), RESTORED_OPERATORS); } private AcknowledgeStreamMockEnvironment createRunAndCheckpointOperatorChain( @@ -229,10 +207,10 @@ private AcknowledgeStreamMockEnvironment createRunAndCheckpointOperatorChain( OneInputStreamOperator tailOperator, Optional stateHandles) throws Exception { - final OneInputStreamTask streamTask = new OneInputStreamTask<>(); final OneInputStreamTaskTestHarness testHarness = - new OneInputStreamTaskTestHarness( - streamTask, 1, 1, + new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + 1, 1, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); @@ -248,12 +226,11 @@ private AcknowledgeStreamMockEnvironment createRunAndCheckpointOperatorChain( new MockInputSplitProvider(), testHarness.bufferSize); - if (stateHandles.isPresent()) { - streamTask.setInitialState(stateHandles.get()); - } - testHarness.invoke(environment); + testHarness.invoke(environment, stateHandles.orElse(null)); testHarness.waitForTaskRunning(); + OneInputStreamTask streamTask = testHarness.getTask(); + processRecords(testHarness); triggerCheckpoint(testHarness, environment, streamTask); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java index 67512b069076f..2cf217fcbed61 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java @@ -50,10 +50,13 @@ public class SourceExternalCheckpointTriggerTest { private static final MultiShotLatch sync = new MultiShotLatch(); @Test + @SuppressWarnings("unchecked") public void testCheckpointsTriggeredBySource() throws Exception { // set up the basic test harness - final SourceStreamTask sourceTask = new SourceStreamTask>(); - final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(sourceTask, BasicTypeInfo.LONG_TYPE_INFO); + final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>( + SourceStreamTask::new, + BasicTypeInfo.LONG_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); testHarness.getExecutionConfig().setLatencyTrackingInterval(-1); @@ -69,6 +72,9 @@ public void testCheckpointsTriggeredBySource() throws Exception { // this starts the source thread testHarness.invoke(); + + final StreamTask sourceTask = testHarness.getTask(); + ready.await(); // now send an external trigger that should be ignored diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskStoppingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskStoppingTest.java index c5fd682e449f0..a58f4369e1f12 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskStoppingTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskStoppingTest.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.functions.StoppableFunction; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.operators.StoppableStreamSource; @@ -33,13 +34,14 @@ */ public class SourceStreamTaskStoppingTest { - // test flag for testStop() static boolean stopped = false; @Test public void testStop() { - final StoppableSourceStreamTask sourceTask = new StoppableSourceStreamTask<>(); + final StoppableSourceStreamTask sourceTask = + new StoppableSourceStreamTask<>(new DummyEnvironment("test", 1, 0), null); + sourceTask.headOperator = new StoppableStreamSource<>(new StoppableSource()); sourceTask.stop(); @@ -50,7 +52,8 @@ public void testStop() { @Test public void testStopBeforeInitialization() throws Exception { - final StoppableSourceStreamTask sourceTask = new StoppableSourceStreamTask<>(); + final StoppableSourceStreamTask sourceTask = + new StoppableSourceStreamTask<>(new DummyEnvironment("test", 1, 0), null); sourceTask.stop(); sourceTask.headOperator = new StoppableStreamSource<>(new StoppableFailingSource()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index f434933572854..f5ac15ddead54 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -56,9 +56,11 @@ public class SourceStreamTaskTest { * This test verifies that open() and close() are correctly called by the StreamTask. */ @Test + @SuppressWarnings("unchecked") public void testOpenClose() throws Exception { - final SourceStreamTask, StreamSource>> sourceTask = new SourceStreamTask<>(); - final StreamTaskTestHarness testHarness = new StreamTaskTestHarness(sourceTask, BasicTypeInfo.STRING_TYPE_INFO); + final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>( + SourceStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -99,10 +101,10 @@ public void testCheckpointing() throws Exception { ExecutorService executor = Executors.newFixedThreadPool(10); try { - final TupleTypeInfo> typeInfo = new TupleTypeInfo>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - final SourceStreamTask, SourceFunction>, - StreamSource, SourceFunction>>> sourceTask = new SourceStreamTask<>(); - final StreamTaskTestHarness> testHarness = new StreamTaskTestHarness>(sourceTask, typeInfo); + final TupleTypeInfo> typeInfo = new TupleTypeInfo<>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + + final StreamTaskTestHarness> testHarness = new StreamTaskTestHarness<>( + SourceStreamTask::new, typeInfo); testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -116,6 +118,9 @@ public void testCheckpointing() throws Exception { // invoke this first, so the tasks are actually running when the checkpoints are scheduled testHarness.invoke(); + testHarness.waitForTaskRunning(); + + final StreamTask, ?> sourceTask = testHarness.getTask(); for (int i = 0; i < numCheckpointers; i++) { checkpointerResults[i] = executor.submit(new Checkpointer(numCheckpoints, checkpointInterval, sourceTask)); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java index 0a563ee354eed..6bd4acc28b1eb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java @@ -22,7 +22,9 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineOnCancellationBarrierException; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.functions.co.CoMapFunction; @@ -33,6 +35,8 @@ import org.junit.Test; +import javax.annotation.Nullable; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -54,13 +58,15 @@ public class StreamTaskCancellationBarrierTest { */ @Test public void testEmitCancellationBarrierWhenNotReady() throws Exception { - StreamTask task = new InitBlockingTask(); - StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(task, BasicTypeInfo.STRING_TYPE_INFO); + StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>( + InitBlockingTask::new, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); // start the test - this cannot succeed across the 'init()' method testHarness.invoke(); + StreamTask task = testHarness.getTask(); + // tell the task to commence a checkpoint boolean result = task.triggerCheckpoint(new CheckpointMetaData(41L, System.currentTimeMillis()), CheckpointOptions.forCheckpoint()); @@ -82,9 +88,8 @@ public void testEmitCancellationBarrierWhenNotReady() throws Exception { @Test public void testDeclineCallOnCancelBarrierOneInput() throws Exception { - OneInputStreamTask task = new OneInputStreamTask(); OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( - task, + OneInputStreamTask::new, 1, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); @@ -120,7 +125,7 @@ public void testDeclineCallOnCancelBarrierOneInput() throws Exception { } /** - * This test verifies (for one input tasks) that the Stream tasks react the following way to + * This test verifies (for two input tasks) that the Stream tasks react the following way to * receiving a checkpoint cancellation barrier: * - send a "decline checkpoint" notification out (to the JobManager) * - emit a cancellation barrier downstream. @@ -128,9 +133,8 @@ public void testDeclineCallOnCancelBarrierOneInput() throws Exception { @Test public void testDeclineCallOnCancelBarrierTwoInputs() throws Exception { - TwoInputStreamTask task = new TwoInputStreamTask(); TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness<>( - task, + TwoInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); @@ -173,6 +177,10 @@ private static class InitBlockingTask extends StreamTask> extends StreamTask { - public BlockingStreamTask() { + public BlockingStreamTask(Environment env, @Nullable TaskStateSnapshot initialState) { + super(env, initialState); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index d907df19f47ae..46862f24c7c5b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -107,6 +107,8 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import javax.annotation.Nullable; + import java.io.Closeable; import java.io.IOException; import java.io.ObjectInputStream; @@ -142,6 +144,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.whenNew; @@ -306,9 +309,8 @@ public void testFailingCheckpointStreamOperator() throws Exception { Environment mockEnvironment = mock(Environment.class); when(mockEnvironment.getTaskInfo()).thenReturn(mockTaskInfo); - StreamTask> streamTask = mock(StreamTask.class, Mockito.CALLS_REAL_METHODS); + StreamTask streamTask = new EmptyStreamTask(mockEnvironment); CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); - streamTask.setEnvironment(mockEnvironment); // mock the operators StreamOperator streamOperator1 = mock(StreamOperator.class); @@ -380,9 +382,8 @@ public void testFailingAsyncCheckpointRunnable() throws Exception { Environment mockEnvironment = mock(Environment.class); when(mockEnvironment.getTaskInfo()).thenReturn(mockTaskInfo); - StreamTask> streamTask = mock(StreamTask.class, Mockito.CALLS_REAL_METHODS); + StreamTask streamTask = spy(new EmptyStreamTask(mockEnvironment)); CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); - streamTask.setEnvironment(mockEnvironment); // mock the operators StreamOperator streamOperator1 = mock(StreamOperator.class); @@ -473,9 +474,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { } }).when(mockEnvironment).acknowledgeCheckpoint(anyLong(), any(CheckpointMetrics.class), any(TaskStateSnapshot.class)); - StreamTask> streamTask = mock(StreamTask.class, Mockito.CALLS_REAL_METHODS); + StreamTask streamTask = new EmptyStreamTask(mockEnvironment); CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); - streamTask.setEnvironment(mockEnvironment); StreamOperator streamOperator = mock(StreamOperator.class); @@ -596,9 +596,8 @@ public OperatorSubtaskState answer(InvocationOnMock invocation) throws Throwable } }); - StreamTask> streamTask = mock(StreamTask.class, Mockito.CALLS_REAL_METHODS); + StreamTask streamTask = new EmptyStreamTask(mockEnvironment); CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); - streamTask.setEnvironment(mockEnvironment); final StreamOperator streamOperator = mock(StreamOperator.class); final OperatorID operatorID = new OperatorID(); @@ -705,9 +704,8 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { when(mockEnvironment.getTaskInfo()).thenReturn(mockTaskInfo); - StreamTask> streamTask = mock(StreamTask.class, Mockito.CALLS_REAL_METHODS); + StreamTask streamTask = new EmptyStreamTask(mockEnvironment); CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); - streamTask.setEnvironment(mockEnvironment); // mock the operators StreamOperator statelessOperator = @@ -802,7 +800,7 @@ public void testOperatorClosingBeforeStopRunning() throws Throwable { private static class NoOpStreamTask> extends StreamTask { public NoOpStreamTask(Environment environment) { - setEnvironment(environment); + super(environment, null); } @Override @@ -1042,6 +1040,25 @@ public AbstractKeyedStateBackend answer(InvocationOnMock invocationOnMock) throw // ------------------------------------------------------------------------ // ------------------------------------------------------------------------ + private static class EmptyStreamTask extends StreamTask> { + + public EmptyStreamTask(Environment env) { + super(env, null); + } + + @Override + protected void init() throws Exception {} + + @Override + protected void run() throws Exception {} + + @Override + protected void cleanup() throws Exception {} + + @Override + protected void cancelTask() throws Exception {} + } + /** * Source that instantiates the operator state backend and the keyed state backend. * The created state backends can be retrieved from the static fields to check if the @@ -1054,6 +1071,10 @@ public static class StateBackendTestSource extends StreamTask> { + public CancelFailingTask(Environment env, @Nullable TaskStateSnapshot initialState) { + super(env, initialState); + } + @Override protected void init() {} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index 3c8dd0b399013..6573ecd290008 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -22,7 +22,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -44,11 +46,16 @@ import org.junit.Assert; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; +import java.util.function.BiFunction; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Test harness for testing a {@link StreamTask}. @@ -63,10 +70,12 @@ */ public class StreamTaskTestHarness { - public static final int DEFAULT_MEMORY_MANAGER_SIZE = 1024 * 1024; + public static final int DEFAULT_MEMORY_MANAGER_SIZE = 1024 * 1024; public static final int DEFAULT_NETWORK_BUFFER_SIZE = 1024; + private final BiFunction> taskFactory; + public long memorySize = 0; public int bufferSize = 0; @@ -76,7 +85,7 @@ public class StreamTaskTestHarness { public Configuration taskConfig; protected StreamConfig streamConfig; - private AbstractInvokable task; + private StreamTask task; private TypeSerializer outputSerializer; private TypeSerializer outputStreamRecordSerializer; @@ -96,8 +105,11 @@ public class StreamTaskTestHarness { @SuppressWarnings("rawtypes") protected StreamTestSingleInputGate[] inputGates; - public StreamTaskTestHarness(AbstractInvokable task, TypeInformation outputType) { - this.task = task; + public StreamTaskTestHarness( + BiFunction> taskFactory, + TypeInformation outputType) { + + this.taskFactory = checkNotNull(taskFactory); this.memorySize = DEFAULT_MEMORY_MANAGER_SIZE; this.bufferSize = DEFAULT_NETWORK_BUFFER_SIZE; @@ -112,10 +124,7 @@ public StreamTaskTestHarness(AbstractInvokable task, TypeInformation output } public ProcessingTimeService getProcessingTimeService() { - if (!(task instanceof StreamTask)) { - throw new UnsupportedOperationException("getProcessingTimeService() only supported on StreamTasks."); - } - return ((StreamTask) task).getProcessingTimeService(); + return task.getProcessingTimeService(); } /** @@ -174,7 +183,18 @@ public StreamMockEnvironment createEnvironment() { * Task thread to finish running. */ public void invoke() throws Exception { - invoke(createEnvironment()); + invoke(createEnvironment(), null); + } + + /** + * Invoke the Task. This resets the output of any previous invocation. This will start a new + * Thread to execute the Task in. Use {@link #waitForTaskCompletion()} to wait for the + * Task thread to finish running. + * + *

Variant for providing initial task state. + */ + public void invoke(TaskStateSnapshot initialState) throws Exception { + invoke(createEnvironment(), initialState); } /** @@ -182,12 +202,22 @@ public void invoke() throws Exception { * Thread to execute the Task in. Use {@link #waitForTaskCompletion()} to wait for the * Task thread to finish running. * - *

Variant for providing a custom environment. + *

Variant for providing a custom environment but no initial state. */ public void invoke(StreamMockEnvironment mockEnv) throws Exception { - this.mockEnv = mockEnv; + invoke(mockEnv, null); + } - task.setEnvironment(mockEnv); + /** + * Invoke the Task. This resets the output of any previous invocation. This will start a new + * Thread to execute the Task in. Use {@link #waitForTaskCompletion()} to wait for the + * Task thread to finish running. + * + *

Variant for providing a custom environment and initial task state. + */ + public void invoke(StreamMockEnvironment mockEnv, @Nullable TaskStateSnapshot initialState) throws Exception { + this.mockEnv = checkNotNull(mockEnv); + this.task = taskFactory.apply(mockEnv, initialState); initializeInputs(); initializeOutput(); @@ -263,6 +293,10 @@ public void waitForTaskRunning(long timeout) throws Exception { } } + public StreamTask getTask() { + return task; + } + /** * Get all the output from the task. This contains StreamRecords and Events interleaved. Use * {@link org.apache.flink.streaming.util.TestHarnessUtil#getRawElementsFromOutput(java.util.Queue)}} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarnessTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarnessTest.java index 249a326292a27..ceea6aec67fec 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarnessTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarnessTest.java @@ -38,7 +38,7 @@ public class StreamTaskTestHarnessTest { public void testMultipleSetupsThrowsException() { StreamTaskTestHarness harness; - harness = new StreamTaskTestHarness<>(new OneInputStreamTask<>(), BasicTypeInfo.STRING_TYPE_INFO); + harness = new StreamTaskTestHarness<>(OneInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO); harness.setupOutputForSingletonOperatorChain(); try { @@ -60,7 +60,7 @@ public void testMultipleSetupsThrowsException() { // expected } - harness = new StreamTaskTestHarness<>(new OneInputStreamTask<>(), BasicTypeInfo.STRING_TYPE_INFO); + harness = new StreamTaskTestHarness<>(OneInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO); harness.setupOperatorChain(new OperatorID(), new TestOperator()) .chain(new OperatorID(), new TestOperator(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig())); @@ -83,7 +83,7 @@ public void testMultipleSetupsThrowsException() { // expected } - harness = new StreamTaskTestHarness<>(new TwoInputStreamTask<>(), BasicTypeInfo.STRING_TYPE_INFO); + harness = new StreamTaskTestHarness<>(TwoInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO); harness.setupOperatorChain(new OperatorID(), new TwoInputTestOperator()) .chain(new OperatorID(), new TestOperator(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig())); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java index ddb5f9b3be77c..bc69e1c6c251b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java @@ -78,6 +78,8 @@ import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.Collections; import java.util.concurrent.Callable; @@ -454,6 +456,10 @@ public void snapshotState(StateSnapshotContext context) throws Exception { */ public static final class TestStreamTask extends OneInputStreamTask { + public TestStreamTask(Environment env, @Nullable TaskStateSnapshot initialState) { + super(env, initialState); + } + @Override public void init() {} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index d2e4dec036892..d1623a22e48cd 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -58,8 +58,10 @@ public class TwoInputStreamTaskTest { @Test @SuppressWarnings("unchecked") public void testOpenCloseAndTimestamps() throws Exception { - final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); - final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness(coMapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final TwoInputStreamTaskTestHarness testHarness = + new TwoInputStreamTaskTestHarness<>( + TwoInputStreamTask::new, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -100,10 +102,11 @@ public void testOpenCloseAndTimestamps() throws Exception { @Test @SuppressWarnings("unchecked") public void testWatermarkAndStreamStatusForwarding() throws Exception { - final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); + final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness( - coMapTask, 2, 2, new int[] {1, 2}, + TwoInputStreamTask::new, + 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); @@ -212,8 +215,11 @@ public void testWatermarkAndStreamStatusForwarding() throws Exception { @Test @SuppressWarnings("unchecked") public void testCheckpointBarriers() throws Exception { - final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); - final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final TwoInputStreamTaskTestHarness testHarness = + new TwoInputStreamTaskTestHarness( + TwoInputStreamTask::new, + 2, 2, new int[] {1, 2}, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); @@ -293,8 +299,12 @@ public void testCheckpointBarriers() throws Exception { @Test @SuppressWarnings("unchecked") public void testOvertakingCheckpointBarriers() throws Exception { - final TwoInputStreamTask coMapTask = new TwoInputStreamTask(); - final TwoInputStreamTaskTestHarness testHarness = new TwoInputStreamTaskTestHarness(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final TwoInputStreamTaskTestHarness testHarness = + new TwoInputStreamTaskTestHarness<>( + TwoInputStreamTask::new, + 2, 2, new int[] {1, 2}, + BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java index 9b9038f846d0e..155e45bbf3003 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.graph.StreamEdge; @@ -31,6 +33,7 @@ import java.io.IOException; import java.util.LinkedList; import java.util.List; +import java.util.function.BiFunction; /** @@ -68,14 +71,16 @@ public class TwoInputStreamTaskTestHarness extends StreamTaskTest * of channels per input gate. Parameter inputGateAssignment specifies for each gate whether * it should be assigned to the first (1), or second (2) input of the task. */ - public TwoInputStreamTaskTestHarness(TwoInputStreamTask task, + public TwoInputStreamTaskTestHarness( + BiFunction> taskFactory, int numInputGates, int numInputChannelsPerGate, int[] inputGateAssignment, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType) { - super(task, outputType); + + super(taskFactory, outputType); this.inputType1 = inputType1; inputSerializer1 = inputType1.createSerializer(executionConfig); @@ -93,11 +98,13 @@ public TwoInputStreamTaskTestHarness(TwoInputStreamTask task, * input gate is assigned to the first task input, the second input gate is assigned to the * second task input. */ - public TwoInputStreamTaskTestHarness(TwoInputStreamTask task, + public TwoInputStreamTaskTestHarness( + BiFunction> taskFactory, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType) { - this(task, 2, 1, new int[] {1, 2}, inputType1, inputType2, outputType); + + this(taskFactory, 2, 1, new int[] {1, 2}, inputType1, inputType2, outputType); } @Override @@ -161,5 +168,10 @@ protected void initializeInputs() throws IOException, InterruptedException { streamConfig.setTypeSerializerIn2(inputSerializer2); } + @Override + @SuppressWarnings("unchecked") + public TwoInputStreamTask getTask() { + return (TwoInputStreamTask) super.getTask(); + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java index 39eeccb1f8e3d..7f8572b560c59 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobRetrievalException; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -139,6 +140,10 @@ public void testNonExistingJobRetrieval() throws Exception { */ public static class SemaphoreInvokable extends AbstractInvokable { + public SemaphoreInvokable(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { lock.acquire(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java index e0bd451d4e3a4..24a24ae64c316 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java @@ -23,6 +23,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.reader.RecordReader; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -73,6 +74,10 @@ public class NetworkStackThroughputITCase extends TestLogger { */ public static class SpeedTestProducer extends AbstractInvokable { + public SpeedTestProducer(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { RecordWriter writer = new RecordWriter<>(getEnvironment().getWriter(0)); @@ -113,6 +118,10 @@ public void invoke() throws Exception { */ public static class SpeedTestForwarder extends AbstractInvokable { + public SpeedTestForwarder(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { RecordReader reader = new RecordReader<>( @@ -143,6 +152,10 @@ public void invoke() throws Exception { */ public static class SpeedTestConsumer extends AbstractInvokable { + public SpeedTestConsumer(Environment environment) { + super(environment); + } + @Override public void invoke() throws Exception { RecordReader reader = new RecordReader<>( diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala index 1883c23cf962c..a970c1de565ec 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala @@ -18,26 +18,21 @@ package org.apache.flink.api.scala.manual -import java.io.File +import java.io._ import java.util.Random -import java.io.BufferedWriter -import java.io.FileWriter import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.common.typeutils.CompositeType +import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory import org.apache.flink.api.scala._ -import java.io.BufferedReader - import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync -import java.io.FileReader - -import org.apache.flink.util.{MutableObjectIterator, TestLogger} import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.runtime.operators.sort.UnilateralSortMerger -import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory +import org.apache.flink.runtime.operators.testutils.DummyInvokable +import org.apache.flink.util.{MutableObjectIterator, TestLogger} + import org.junit.Assert._ -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable /** * This test is wrote as manual test. @@ -99,7 +94,7 @@ class MassiveCaseClassSortingITCase extends TestLogger { val ioMan = new IOManagerAsync() sorter = new UnilateralSortMerger[StringTuple](mm, ioMan, inputIterator, - new DummyInvokable(), + new DummyInvokable(), new RuntimeSerializerFactory[StringTuple](serializer, classOf[StringTuple]), comparator, 1.0, 4, 0.8f, true /*use large record handler*/, false) @@ -238,8 +233,3 @@ class StringTupleReader(val reader: BufferedReader) extends MutableObjectIterato } } - -class DummyInvokable extends AbstractInvokable { - - override def invoke() = {} -} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITCase.scala index 85747d5dcda87..c0ccd53c87526 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITCase.scala @@ -25,6 +25,7 @@ import akka.actor.ActorSystem import akka.testkit.{ImplicitSender, TestKit} import org.apache.flink.core.testutils.OneShotLatch import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} +import org.apache.flink.runtime.execution.Environment import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex} import org.apache.flink.runtime.messages.JobManagerMessages._ @@ -83,7 +84,7 @@ class JobManagerLeaderSessionIDITCase(_system: ActorSystem) } } -class BlockingUntilSignalNoOpInvokable extends AbstractInvokable { +class BlockingUntilSignalNoOpInvokable(env: Environment) extends AbstractInvokable(env) { override def invoke(): Unit = { BlockingUntilSignalNoOpInvokable.countDownLatch.countDown()