From 8128daf3472c80646a1c4208b809b5f992ab1f82 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 3 Feb 2016 11:07:38 +0100 Subject: [PATCH 1/3] [hotfix] Fix typos in Trigger.java --- .../flink/api/common/ExecutionConfig.java | 6 +- .../org/apache/flink/api/common/Plan.java | 11 -- .../plantranslate/JobGraphGenerator.java | 16 +- .../BackPressureStatsTrackerITCase.java | 3 +- .../StackTraceSampleCoordinatorITCase.java | 3 +- .../deployment/TaskDeploymentDescriptor.java | 30 ++- .../flink/runtime/execution/Environment.java | 8 + .../executiongraph/ExecutionGraph.java | 18 +- .../executiongraph/ExecutionVertex.java | 4 +- .../flink/runtime/jobgraph/JobGraph.java | 88 +++++---- .../jobgraph/tasks/AbstractInvokable.java | 27 +-- .../flink/runtime/operators/DataSinkTask.java | 20 +- .../runtime/operators/DataSourceTask.java | 18 +- .../taskmanager/RuntimeEnvironment.java | 11 +- .../flink/runtime/taskmanager/Task.java | 10 +- .../flink/runtime/jobmanager/JobManager.scala | 3 +- .../checkpoint/CoordinatorShutdownTest.java | 5 +- ...ecutionGraphCheckpointCoordinatorTest.java | 20 +- .../client/JobClientActorRecoveryITCase.java | 3 +- .../runtime/client/JobClientActorTest.java | 3 +- .../TaskDeploymentDescriptorTest.java | 12 +- .../ExecutionGraphConstructionTest.java | 28 ++- .../ExecutionGraphDeploymentTest.java | 7 +- .../ExecutionGraphRestartTest.java | 36 ++-- .../ExecutionGraphSignalsTest.java | 2 + .../ExecutionGraphTestUtils.java | 4 +- .../ExecutionStateProgressTest.java | 172 +++++++++--------- .../executiongraph/LocalInputSplitsTest.java | 7 +- .../executiongraph/PointwisePatternTest.java | 20 +- .../TerminalStateDeadlockTest.java | 2 + .../VertexLocationConstraintTest.java | 19 +- .../executiongraph/VertexSlotSharingTest.java | 2 + .../PartialConsumePipelinedResultTest.java | 3 +- .../flink/runtime/jobgraph/JobGraphTest.java | 25 +-- .../jobgraph/jsonplan/JsonGeneratorTest.java | 5 +- .../runtime/jobmanager/JobManagerTest.java | 7 +- .../runtime/jobmanager/JobSubmitTest.java | 5 +- .../SlotCountExceedingParallelismTest.java | 3 +- .../StandaloneSubmittedJobGraphStoreTest.java | 3 +- ...ooKeeperSubmittedJobGraphsStoreITCase.java | 3 +- .../ScheduleOrUpdateConsumersTest.java | 2 + .../LeaderChangeStateCleanupTest.java | 3 +- .../operators/testutils/DummyEnvironment.java | 7 + .../operators/testutils/MockEnvironment.java | 11 +- .../taskmanager/TaskAsyncCallTest.java | 3 +- .../runtime/taskmanager/TaskCancelTest.java | 3 +- .../runtime/taskmanager/TaskManagerTest.java | 45 +++-- .../runtime/taskmanager/TaskStopTest.java | 2 + .../flink/runtime/taskmanager/TaskTest.java | 3 +- .../TaskManagerLossFailsTasksTest.scala | 5 +- .../CoLocationConstraintITCase.scala | 3 +- .../runtime/jobmanager/JobManagerITCase.scala | 51 +++--- .../runtime/jobmanager/RecoveryITCase.scala | 16 +- .../jobmanager/SlotSharingITCase.scala | 7 +- ...askManagerFailsWithSlotSharingITCase.scala | 6 +- .../api/graph/StreamingJobGraphGenerator.java | 16 +- .../streaming/api/RestartStrategyTest.java | 6 +- .../graph/StreamingJobGraphGeneratorTest.java | 20 +- .../partitioner/RescalePartitionerTest.java | 2 + .../runtime/tasks/StreamMockEnvironment.java | 18 +- .../tasks/StreamTaskAsyncCheckpointTest.java | 6 + .../runtime/tasks/StreamTaskTest.java | 3 +- .../runtime/tasks/StreamTaskTestHarness.java | 10 +- .../JobSubmissionFailsITCase.java | 7 +- .../test/javaApiOperators/MapITCase.java | 33 ++++ .../JobManagerHACheckpointRecoveryITCase.java | 3 +- .../JobManagerHAJobGraphRecoveryITCase.java | 3 +- .../runtime/NetworkStackThroughputITCase.java | 3 +- .../ZooKeeperLeaderElectionITCase.java | 3 +- .../flink/test/web/WebFrontendITCase.java | 5 +- .../jobmanager/JobManagerFailsITCase.scala | 5 +- .../JobManagerLeaderSessionIDITSuite.scala | 3 +- .../taskmanager/TaskManagerFailsITCase.scala | 11 +- 73 files changed, 565 insertions(+), 431 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index ffd30e98613ce..2099b3d95b02b 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -59,9 +59,6 @@ public class ExecutionConfig implements Serializable { private static final long serialVersionUID = 1L; - // Key for storing it in the Job Configuration - public static final String CONFIG_KEY = "runtime.config"; - /** * The constant to use for the parallelism, if the system should use the number * of currently available slots. @@ -650,7 +647,8 @@ public boolean equals(Object obj) { Objects.equals(executionMode, other.executionMode) && useClosureCleaner == other.useClosureCleaner && parallelism == other.parallelism && - restartStrategyConfiguration.equals(other.restartStrategyConfiguration) && + ((restartStrategyConfiguration == null && other.restartStrategyConfiguration == null) || + restartStrategyConfiguration.equals(other.restartStrategyConfiguration)) && forceKryo == other.forceKryo && objectReuse == other.objectReuse && autoTypeRegistrationEnabled == other.autoTypeRegistrationEnabled && diff --git a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java index d81fcd15ab24f..3e5cb618a6fa4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java @@ -38,7 +38,6 @@ import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; import org.apache.flink.api.common.operators.GenericDataSinkBase; import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.util.Visitable; @@ -294,16 +293,6 @@ public void setDefaultParallelism(int defaultParallelism) { this.defaultParallelism = defaultParallelism; } - /** - * Returns the specified restart strategy configuration. This configuration defines the used - * restart strategy to be used at runtime. - * - * @return The specified restart strategy configuration - */ - public RestartStrategies.RestartStrategyConfiguration getRestartStrategyConfiguration() { - return getExecutionConfig().getRestartStrategy(); - } - /** * Gets the optimizer post-pass class for this job. The post-pass typically creates utility classes * for data types and is specific to a particular data model (record, tuple, Scala, ...) diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java index f59c347667510..c7aaa7d0d856d 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.core.JsonFactory; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.aggregators.AggregatorRegistry; import org.apache.flink.api.common.aggregators.AggregatorWithName; @@ -81,11 +80,9 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.StringUtils; import org.apache.flink.util.Visitor; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -217,9 +214,7 @@ public JobGraph compileJobGraph(OptimizedPlan program, JobID jobId) { // ----------- finalize the job graph ----------- // create the job graph object - JobGraph graph = new JobGraph(jobId, program.getJobName()); - - graph.setRestartStrategyConfiguration(program.getOriginalPlan().getRestartStrategyConfiguration()); + JobGraph graph = new JobGraph(jobId, program.getJobName(), program.getOriginalPlan().getExecutionConfig()); graph.setAllowQueuedScheduling(false); graph.setSessionTimeout(program.getOriginalPlan().getSessionTimeout()); @@ -238,15 +233,6 @@ public JobGraph compileJobGraph(OptimizedPlan program, JobID jobId) { DistributedCache.writeFileInfoToConfig(e.getKey(), e.getValue(), graph.getJobConfiguration()); } - try { - InstantiationUtil.writeObjectToConfig( - program.getOriginalPlan().getExecutionConfig(), - graph.getJobConfiguration(), - ExecutionConfig.CONFIG_KEY); - } catch (IOException e) { - throw new RuntimeException("Config object could not be written to Job Configuration: " + e); - } - // release all references again this.vertices = null; this.chainedTasks = null; diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java index 26591857d8552..1f0b2effe903b 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/BackPressureStatsTrackerITCase.java @@ -20,6 +20,7 @@ import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; @@ -92,7 +93,7 @@ public void testBackPressuredProducer() throws Exception { final FiniteDuration deadline = new FiniteDuration(60, TimeUnit.SECONDS); // The JobGraph - final JobGraph jobGraph = new JobGraph(); + final JobGraph jobGraph = new JobGraph(new ExecutionConfig()); final int parallelism = 4; final JobVertex task = new JobVertex("Task"); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java index 63c3712e2b50b..c6ce31573ba70 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/StackTraceSampleCoordinatorITCase.java @@ -20,6 +20,7 @@ import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -76,7 +77,7 @@ public void testTaskClearedWhileSampling() throws Exception { final FiniteDuration deadline = new FiniteDuration(60, TimeUnit.SECONDS); // The JobGraph - final JobGraph jobGraph = new JobGraph(); + final JobGraph jobGraph = new JobGraph(new ExecutionConfig()); final int parallelism = 1; final JobVertex task = new JobVertex("Task"); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 983ad38bb6e10..60b8ba65b1fce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.deployment; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; @@ -88,6 +89,9 @@ public final class TaskDeploymentDescriptor implements Serializable { private final SerializedValue> operatorState; + /** The execution configuration (see {@link ExecutionConfig}) related to the specific job. */ + private final ExecutionConfig executionConfig; + private long recoveryTimestamp; /** @@ -95,9 +99,9 @@ public final class TaskDeploymentDescriptor implements Serializable { */ public TaskDeploymentDescriptor( JobID jobID, JobVertexID vertexID, ExecutionAttemptID executionId, - String taskName, int indexInSubtaskGroup, int numberOfSubtasks, int attemptNumber, - Configuration jobConfiguration, Configuration taskConfiguration, String invokableClassName, - List producedPartitions, + ExecutionConfig executionConfig, String taskName, int indexInSubtaskGroup, int numberOfSubtasks, + int attemptNumber, Configuration jobConfiguration, Configuration taskConfiguration, + String invokableClassName, List producedPartitions, List inputGates, List requiredJarFiles, List requiredClasspaths, int targetSlotNumber, SerializedValue> operatorState, @@ -111,6 +115,7 @@ public TaskDeploymentDescriptor( this.jobID = checkNotNull(jobID); this.vertexID = checkNotNull(vertexID); this.executionId = checkNotNull(executionId); + this.executionConfig = checkNotNull(executionConfig); this.taskName = checkNotNull(taskName); this.indexInSubtaskGroup = indexInSubtaskGroup; this.numberOfSubtasks = numberOfSubtasks; @@ -129,16 +134,23 @@ public TaskDeploymentDescriptor( public TaskDeploymentDescriptor( JobID jobID, JobVertexID vertexID, ExecutionAttemptID executionId, - String taskName, int indexInSubtaskGroup, int numberOfSubtasks, int attemptNumber, - Configuration jobConfiguration, Configuration taskConfiguration, String invokableClassName, - List producedPartitions, + ExecutionConfig executionConfig, String taskName, int indexInSubtaskGroup, int numberOfSubtasks, + int attemptNumber, Configuration jobConfiguration, Configuration taskConfiguration, + String invokableClassName, List producedPartitions, List inputGates, List requiredJarFiles, List requiredClasspaths, int targetSlotNumber) { - this(jobID, vertexID, executionId, taskName, indexInSubtaskGroup, numberOfSubtasks, attemptNumber, - jobConfiguration, taskConfiguration, invokableClassName, producedPartitions, - inputGates, requiredJarFiles, requiredClasspaths, targetSlotNumber, null, -1); + this(jobID, vertexID, executionId, executionConfig, taskName, indexInSubtaskGroup, + numberOfSubtasks, attemptNumber, jobConfiguration, taskConfiguration, invokableClassName, + producedPartitions, inputGates, requiredJarFiles, requiredClasspaths, targetSlotNumber, null, -1); + } + + /** + * Returns the execution configuration (see {@link ExecutionConfig}) related to the specific job. + */ + public ExecutionConfig getExecutionConfig() { + return executionConfig; } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index 73321516ba055..a10c463a74440 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.execution; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; @@ -45,6 +46,13 @@ */ public interface Environment { + /** + * Returns the job specific {@link ExecutionConfig}. + * + * @return The execution configuration associated with the current job. + * */ + ExecutionConfig getExecutionConfig(); + /** * Returns the ID of the job that the task belongs to. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index ed50bea8f06fe..9fda02c00414e 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -20,6 +20,7 @@ import akka.actor.ActorSystem; +import com.google.common.base.Preconditions; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; @@ -60,7 +61,6 @@ import org.apache.flink.runtime.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -180,6 +180,9 @@ public class ExecutionGraph implements Serializable { // ------ Configuration of the Execution ------- + /** The execution configuration (see {@link ExecutionConfig}) related to this specific job. */ + private ExecutionConfig executionConfig; + /** Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able * to deploy them immediately. */ private boolean allowQueuedScheduling = false; @@ -234,7 +237,6 @@ public class ExecutionGraph implements Serializable { private ExecutionContext executionContext; // ------ Fields that are only relevant for archived execution graphs ------------ - private ExecutionConfig executionConfig; private String jsonPlan; @@ -250,6 +252,7 @@ public class ExecutionGraph implements Serializable { JobID jobId, String jobName, Configuration jobConfig, + ExecutionConfig config, FiniteDuration timeout, RestartStrategy restartStrategy) { this( @@ -257,6 +260,7 @@ public class ExecutionGraph implements Serializable { jobId, jobName, jobConfig, + config, timeout, restartStrategy, new ArrayList(), @@ -270,6 +274,7 @@ public ExecutionGraph( JobID jobId, String jobName, Configuration jobConfig, + ExecutionConfig config, FiniteDuration timeout, RestartStrategy restartStrategy, List requiredJarFiles, @@ -302,7 +307,7 @@ public ExecutionGraph( this.requiredJarFiles = requiredJarFiles; this.requiredClasspaths = requiredClasspaths; - + this.executionConfig = Preconditions.checkNotNull(config); this.timeout = timeout; this.restartStrategy = restartStrategy; @@ -942,12 +947,7 @@ public void prepareForArchiving() { if (!state.isTerminalState()) { throw new IllegalStateException("Can only archive the job from a terminal state"); } - // "unpack" execution config before we throw away the usercode classloader. - try { - executionConfig = (ExecutionConfig) InstantiationUtil.readObjectFromConfig(jobConfiguration, ExecutionConfig.CONFIG_KEY,userClassLoader); - } catch (Exception e) { - LOG.warn("Error deserializing the execution config while archiving the execution graph", e); - } + // clear the non-serializable fields userClassLoader = null; scheduler = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index f2d30b53937f9..80430bce5f4d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.executiongraph; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; @@ -665,10 +666,11 @@ TaskDeploymentDescriptor createDeploymentDescriptor( consumedPartitions.add(new InputGateDeploymentDescriptor(resultId, queueToRequest, partitions)); } + ExecutionConfig config = getExecutionGraph().getExecutionConfig(); List jarFiles = getExecutionGraph().getRequiredJarFiles(); List classpaths = getExecutionGraph().getRequiredClasspaths(); - return new TaskDeploymentDescriptor(getJobId(), getJobvertexId(), executionId, getTaskName(), + return new TaskDeploymentDescriptor(getJobId(), getJobvertexId(), executionId, config, getTaskName(), subTaskIndex, getTotalNumberOfParallelSubtasks(), attemptNumber, getExecutionGraph().getJobConfiguration(), jobVertex.getJobVertex().getConfiguration(), jobVertex.getJobVertex().getInvokableClassName(), producedPartitions, consumedPartitions, jarFiles, classpaths, targetSlot.getRoot().getSlotNumber(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index f99d7542e6009..80488f629be67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.jobgraph; +import com.google.common.base.Preconditions; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; @@ -33,15 +35,14 @@ import java.net.InetSocketAddress; import java.net.URL; import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; import java.util.LinkedHashMap; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Collections; import java.util.Set; +import java.util.LinkedHashSet; +import java.util.Iterator; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; /** * The JobGraph represents a Flink dataflow program, at the low level that the JobManager accepts. * All programs from higher level APIs are transformed into JobGraphs. @@ -62,16 +63,16 @@ public class JobGraph implements Serializable { // -------------------------------------------------------------------------------------------- /** List of task vertices included in this job graph. */ - private final Map taskVertices = new LinkedHashMap(); + private final Map taskVertices = new LinkedHashMap<>(); /** The job configuration attached to this job. */ private final Configuration jobConfiguration = new Configuration(); /** Set of JAR files required to run this job. */ - private final List userJars = new ArrayList(); + private final List userJars = new ArrayList<>(); /** Set of blob keys identifying the JAR files required to run this job. */ - private final List userJarBlobKeys = new ArrayList(); + private final List userJarBlobKeys = new ArrayList<>(); /** ID of this job. May be set if specific job id is desired (e.g. session management) */ private final JobID jobID; @@ -79,9 +80,7 @@ public class JobGraph implements Serializable { /** Name of this job. */ private final String jobName; - /** Configuration which defines which restart strategy to use for the job recovery */ - private RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration; - + private final ExecutionConfig executionConfig; /** The number of seconds after which the corresponding ExecutionGraph is removed at the * job manager after it has been executed. */ @@ -102,60 +101,74 @@ public class JobGraph implements Serializable { // -------------------------------------------------------------------------------------------- /** - * Constructs a new job graph with no name and a random job ID. + * Constructs a new job graph with no name, a random job ID, and the given + * {@link ExecutionConfig}. + * + * @param config The {@link ExecutionConfig} for the job. */ - public JobGraph() { - this((String) null); + public JobGraph(ExecutionConfig config) { + this((String) null, config); } /** - * Constructs a new job graph with the given name, a random job ID. + * Constructs a new job graph with the given name, the given {@link ExecutionConfig}, + * and a random job ID. * - * @param jobName The name of the job + * @param jobName The name of the job. + * @param config The execution configuration of the job. */ - public JobGraph(String jobName) { - this(null, jobName); + public JobGraph(String jobName, ExecutionConfig config) { + this(null, jobName, config); } /** - * Constructs a new job graph with the given name and a random job ID if null supplied as an id. + * Constructs a new job graph with the given job ID (or a random ID, if {@code null} is passed), + * the given name and the given execution configuration (see {@link ExecutionConfig}). * * @param jobId The id of the job. A random ID is generated, if {@code null} is passed. * @param jobName The name of the job. + * @param config The execution configuration of the job. */ - public JobGraph(JobID jobId, String jobName) { + public JobGraph(JobID jobId, String jobName, ExecutionConfig config) { this.jobID = jobId == null ? new JobID() : jobId; this.jobName = jobName == null ? "(unnamed job)" : jobName; + this.executionConfig = Preconditions.checkNotNull(config); } /** - * Constructs a new job graph with no name and a random job ID if null supplied as an id. + * Constructs a new job graph with no name, a random job ID, the given {@link ExecutionConfig}, and + * the given job vertices. * + * @param config The execution configuration of the job. * @param vertices The vertices to add to the graph. */ - public JobGraph(JobVertex... vertices) { - this(null, vertices); + public JobGraph(ExecutionConfig config, JobVertex... vertices) { + this(null, config, vertices); } /** - * Constructs a new job graph with the given name and a random job ID. + * Constructs a new job graph with the given name, the given {@link ExecutionConfig}, a random job ID, + * and the given job vertices. * * @param jobName The name of the job. + * @param config The execution configuration of the job. * @param vertices The vertices to add to the graph. */ - public JobGraph(String jobName, JobVertex... vertices) { - this(null, jobName, vertices); + public JobGraph(String jobName, ExecutionConfig config, JobVertex... vertices) { + this(null, jobName, config, vertices); } /** - * Constructs a new job graph with the given name and a random job ID if null supplied as an id. + * Constructs a new job graph with the given name, the given {@link ExecutionConfig}, + * the given jobId or a random one if null supplied, and the given job vertices. * * @param jobId The id of the job. A random ID is generated, if {@code null} is passed. * @param jobName The name of the job. + * @param config The execution configuration of the job. * @param vertices The vertices to add to the graph. */ - public JobGraph(JobID jobId, String jobName, JobVertex... vertices) { - this(jobId, jobName); + public JobGraph(JobID jobId, String jobName, ExecutionConfig config, JobVertex... vertices) { + this(jobId, jobName, config); for (JobVertex vertex : vertices) { addVertex(vertex); @@ -192,23 +205,8 @@ public Configuration getJobConfiguration() { return this.jobConfiguration; } - /** - * Sets the restart strategy configuration. This configuration specifies the restart strategy - * to be used by the ExecutionGraph in case of a restart. - * - * @param restartStrategyConfiguration Restart strategy configuration to be set - */ - public void setRestartStrategyConfiguration(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { - this.restartStrategyConfiguration = restartStrategyConfiguration; - } - - /** - * Gets the restart strategy configuration - * - * @return Restart strategy configuration to be used - */ - public RestartStrategies.RestartStrategyConfiguration getRestartStrategyConfiguration() { - return restartStrategyConfiguration; + public ExecutionConfig getExecutionConfig() { + return this.executionConfig; } /** 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 30f32a11614fa..d7dfaf5874b53 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 @@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.BatchTask; -import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,9 +41,6 @@ public abstract class AbstractInvokable { /** The environment assigned to this invokable. */ private Environment environment; - /** The execution config, cached from the deserialization from the JobConfiguration */ - private ExecutionConfig executionConfig; - /** * Starts the execution. * @@ -125,29 +121,10 @@ public Configuration getJobConfiguration() { } /** - * Returns the global ExecutionConfig, obtained from the job configuration. + * Returns the global ExecutionConfig. */ public ExecutionConfig getExecutionConfig() { - if (executionConfig != null) { - return executionConfig; - } - - try { - executionConfig = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( - getJobConfiguration(), - ExecutionConfig.CONFIG_KEY, - getUserCodeClassLoader()); - - if (executionConfig == null) { - LOG.warn("Environment did not contain an ExecutionConfig - using a default config."); - executionConfig = new ExecutionConfig(); - } - return executionConfig; - } - catch (Exception e) { - LOG.warn("Could not load ExecutionConfig from Environment, returning default ExecutionConfig", e); - return new ExecutionConfig(); - } + return this.environment.getExecutionConfig(); } /** 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 6972e1d78afc4..21e87843bbd74 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 @@ -41,13 +41,10 @@ import org.apache.flink.runtime.operators.util.ReaderIterator; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.plugable.DeserializationDelegate; -import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.MutableObjectIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * DataSinkTask which is executed by a task manager. The task hands the data to an output format. * @@ -112,21 +109,8 @@ public void invoke() throws Exception { LOG.debug(getLogString("Rich Sink detected. Initializing runtime context.")); } - ExecutionConfig executionConfig; - try { - ExecutionConfig c = InstantiationUtil.readObjectFromConfig( - getJobConfiguration(), - ExecutionConfig.CONFIG_KEY, - getUserCodeClassLoader()); - if (c != null) { - executionConfig = c; - } else { - LOG.warn("The execution config returned by the configuration was null"); - executionConfig = new ExecutionConfig(); - } - } catch (IOException | ClassNotFoundException e) { - throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); - } + ExecutionConfig executionConfig = getExecutionConfig(); + boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled(); try { 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 93a4f9c03b1f4..960faf74f05e0 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 @@ -36,11 +36,9 @@ import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; -import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -104,21 +102,7 @@ public void invoke() throws Exception { LOG.debug(getLogString("Rich Source detected. Initializing runtime context.")); } - ExecutionConfig executionConfig; - try { - ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( - getJobConfiguration(), - ExecutionConfig.CONFIG_KEY, - getUserCodeClassLoader()); - if (c != null) { - executionConfig = c; - } else { - LOG.warn("ExecutionConfig from job configuration is null. Creating empty config"); - executionConfig = new ExecutionConfig(); - } - } catch (IOException | ClassNotFoundException e) { - throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: ", e); - } + ExecutionConfig executionConfig = getExecutionConfig(); boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index 0fddde4980010..51e7e34b3064f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.taskmanager; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; @@ -55,7 +56,8 @@ public class RuntimeEnvironment implements Environment { private final Configuration jobConfiguration; private final Configuration taskConfiguration; - + private final ExecutionConfig executionConfig; + private final ClassLoader userCodeClassLoader; private final MemoryManager memManager; @@ -80,6 +82,7 @@ public RuntimeEnvironment( JobID jobId, JobVertexID jobVertexId, ExecutionAttemptID executionId, + ExecutionConfig executionConfig, TaskInfo taskInfo, Configuration jobConfiguration, Configuration taskConfiguration, @@ -99,6 +102,7 @@ public RuntimeEnvironment( this.jobVertexId = checkNotNull(jobVertexId); this.executionId = checkNotNull(executionId); this.taskInfo = checkNotNull(taskInfo); + this.executionConfig = checkNotNull(executionConfig); this.jobConfiguration = checkNotNull(jobConfiguration); this.taskConfiguration = checkNotNull(taskConfiguration); this.userCodeClassLoader = checkNotNull(userCodeClassLoader); @@ -116,6 +120,11 @@ public RuntimeEnvironment( // ------------------------------------------------------------------------ + @Override + public ExecutionConfig getExecutionConfig() { + return this.executionConfig; + } + @Override public JobID getJobID() { return jobId; 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 f2d60250e5c5f..58e2153cea283 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 @@ -18,6 +18,8 @@ package org.apache.flink.runtime.taskmanager; + +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.cache.DistributedCache; @@ -217,6 +219,9 @@ public class Task implements Runnable { private volatile long recoveryTs; + /** The job specific execution configuration (see {@link ExecutionConfig}). */ + private final ExecutionConfig executionConfig; + /** *

IMPORTANT: This constructor may not start any work that would need to * be undone in the case of a failing task deployment.

@@ -245,6 +250,7 @@ public Task(TaskDeploymentDescriptor tdd, this.nameOfInvokableClass = checkNotNull(tdd.getInvokableClassName()); this.operatorState = tdd.getOperatorState(); this.recoveryTs = tdd.getRecoveryTimestamp(); + this.executionConfig = checkNotNull(tdd.getExecutionConfig()); this.memoryManager = checkNotNull(memManager); this.ioManager = checkNotNull(ioManager); @@ -493,8 +499,8 @@ else if (current == ExecutionState.CANCELING) { TaskInputSplitProvider splitProvider = new TaskInputSplitProvider(jobManager, jobId, vertexId, executionId, userCodeClassLoader, actorAskTimeout); - Environment env = new RuntimeEnvironment(jobId, vertexId, executionId, taskInfo, - jobConfiguration, taskConfiguration, + Environment env = new RuntimeEnvironment(jobId, vertexId, executionId, + executionConfig, taskInfo, jobConfiguration, taskConfiguration, userCodeClassLoader, memoryManager, ioManager, broadcastVariableManager, accumulatorRegistry, splitProvider, distributedCacheEntries, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 1c6fce857473f..1b08e307977ae 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -944,7 +944,7 @@ class JobManager( throw new JobSubmissionException(jobId, "The given job is empty") } - val restartStrategy = Option(jobGraph.getRestartStrategyConfiguration()) + val restartStrategy = Option(jobGraph.getExecutionConfig.getRestartStrategy()) .map(RestartStrategyFactory.createRestartStrategy(_)) match { case Some(strategy) => strategy case None => defaultRestartStrategy @@ -963,6 +963,7 @@ class JobManager( jobGraph.getJobID, jobGraph.getName, jobGraph.getJobConfiguration, + jobGraph.getExecutionConfig, timeout, restartStrategy, jobGraph.getUserJarBlobKeys, 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 1c666e5e5fb13..03ff83d536b9a 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 @@ -18,6 +18,7 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.ListeningBehaviour; @@ -60,7 +61,7 @@ public void testCoordinatorShutsDownOnFailure() { vertex.setInvokableClass(Tasks.NoOpInvokable.class); List vertexIdList = Collections.singletonList(vertex.getID()); - JobGraph testGraph = new JobGraph("test job", vertex); + JobGraph testGraph = new JobGraph("test job", new ExecutionConfig(), vertex); testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList, 5000, 60000, 0L, Integer.MAX_VALUE)); @@ -112,7 +113,7 @@ public void testCoordinatorShutsDownOnSuccess() { vertex.setInvokableClass(Tasks.NoOpInvokable.class); List vertexIdList = Collections.singletonList(vertex.getID()); - JobGraph testGraph = new JobGraph("test job", vertex); + JobGraph testGraph = new JobGraph("test job", new ExecutionConfig(), vertex); testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList, 5000, 60000, 0L, Integer.MAX_VALUE)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java index 5d83bf21ac7d0..b375802fdd7f2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import akka.actor.ActorSystem; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -44,15 +45,16 @@ public class ExecutionGraphCheckpointCoordinatorTest { @Test public void testCheckpointAndSavepointCoordinatorShareCheckpointIDCounter() throws Exception { ExecutionGraph executionGraph = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - new JobID(), - "test", - new Configuration(), - new FiniteDuration(1, TimeUnit.DAYS), - new NoRestartStrategy(), - Collections.emptyList(), - Collections.emptyList(), - ClassLoader.getSystemClassLoader()); + TestingUtils.defaultExecutionContext(), + new JobID(), + "test", + new Configuration(), + new ExecutionConfig(), + new FiniteDuration(1, TimeUnit.DAYS), + new NoRestartStrategy(), + Collections.emptyList(), + Collections.emptyList(), + ClassLoader.getSystemClassLoader()); ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); 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 aeb521caaf2e9..865760e686c5d 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 @@ -20,6 +20,7 @@ import akka.actor.PoisonPill; import org.apache.curator.test.TestingServer; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -94,7 +95,7 @@ public void testJobClientRecovery() throws Exception { JobVertex blockingVertex = new JobVertex("Blocking Vertex"); blockingVertex.setInvokableClass(BlockingTask.class); blockingVertex.setParallelism(1); - final JobGraph jobGraph = new JobGraph("Blocking Test Job", blockingVertex); + final JobGraph jobGraph = new JobGraph("Blocking Test Job", new ExecutionConfig(), blockingVertex); final Promise promise = new scala.concurrent.impl.Promise.DefaultPromise<>(); Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java index 00ad632b343e7..ee1fd60353302 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java @@ -25,6 +25,7 @@ import akka.pattern.Patterns; import akka.testkit.JavaTestKit; import akka.util.Timeout; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.FlinkUntypedActor; @@ -47,7 +48,7 @@ public class JobClientActorTest extends TestLogger { private static ActorSystem system; - private static JobGraph testJobGraph = new JobGraph("Test Job"); + private static JobGraph testJobGraph = new JobGraph("Test Job", new ExecutionConfig()); @BeforeClass public static void setup() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 92b642adc2954..e839c97e34fb5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -54,10 +55,12 @@ public void testSerialization() { final List inputGates = new ArrayList(0); final List requiredJars = new ArrayList(0); final List requiredClasspaths = new ArrayList(0); - - final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, execId, taskName, - indexInSubtaskGroup, currentNumberOfSubtasks, attemptNumber, jobConfiguration, taskConfiguration, - invokableClass.getName(), producedResults, inputGates, requiredJars, requiredClasspaths, 47); + final ExecutionConfig executionConfig = new ExecutionConfig(); + + final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, execId, + executionConfig, taskName, indexInSubtaskGroup, currentNumberOfSubtasks, attemptNumber, + jobConfiguration, taskConfiguration, invokableClass.getName(), producedResults, inputGates, + requiredJars, requiredClasspaths, 47); final TaskDeploymentDescriptor copy = CommonTestUtils.createCopySerializable(orig); @@ -75,6 +78,7 @@ public void testSerialization() { assertEquals(orig.getAttemptNumber(), copy.getAttemptNumber()); assertEquals(orig.getProducedPartitions(), copy.getProducedPartitions()); assertEquals(orig.getInputGates(), copy.getInputGates()); + assertEquals(orig.getExecutionConfig(), copy.getExecutionConfig()); assertEquals(orig.getRequiredJarFiles(), copy.getRequiredJarFiles()); assertEquals(orig.getRequiredClasspaths(), copy.getRequiredClasspaths()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java index ee372dd188e62..f3c1d38e7d0c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java @@ -26,6 +26,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -106,6 +107,7 @@ public void testCreateSimpleGraphBipartite() { jobId, jobName, cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -149,7 +151,8 @@ public void testAttachViaDataSets() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -216,7 +219,8 @@ public void testAttachViaIds() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -470,7 +474,8 @@ public void testCannotConnectMissingId() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -526,7 +531,8 @@ public void testCannotConnectWrongOrder() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -587,7 +593,8 @@ public void testSetupInputSplits() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -632,7 +639,8 @@ public void testMoreThanOneConsumerForIntermediateResult() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); @@ -696,14 +704,16 @@ public void testCoLocationConstraintCreation() { // isolated vertex JobVertex v8 = new JobVertex("vertex8"); v8.setParallelism(2); - - JobGraph jg = new JobGraph(jobId, jobName, v1, v2, v3, v4, v5, v6, v7, v8); + + JobGraph jg = new JobGraph(jobId, jobName, new ExecutionConfig(), + v1, v2, v3, v4, v5, v6, v7, v8); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index 6362732697a50..6d63d74d2e3c7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -83,7 +84,8 @@ public void testBuildDeploymentDescriptor() { TestingUtils.defaultExecutionContext(), jobId, "some job", - new Configuration(), + new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); @@ -287,6 +289,7 @@ private Map setupExecution(JobVertex v1, int dop1 jobId, "some job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); @@ -329,4 +332,4 @@ public void finalizeOnMaster(ClassLoader cl) throws Exception { throw new Exception(); } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index b1f11fbb82cc0..360711a43772e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.executiongraph; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.configuration.Configuration; @@ -72,13 +73,14 @@ public void testNoManualRestart() throws Exception { sender.setInvokableClass(Tasks.NoOpInvokable.class); sender.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("Pointwise job", sender); + JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -127,14 +129,15 @@ public void testConstraintsAfterRestart() throws Exception { groupVertex.setStrictlyCoLocatedWith(groupVertex2); //initiate and schedule job - JobGraph jobGraph = new JobGraph("Pointwise job", groupVertex, groupVertex2); + JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), groupVertex, groupVertex2); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), - AkkaUtils.getDefaultTimeout(), - new FixedDelayRestartStrategy(1, 0L)); + new ExecutionConfig(), + AkkaUtils.getDefaultTimeout(), + new FixedDelayRestartStrategy(1, 0L)); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); assertEquals(JobStatus.CREATED, eg.getState()); @@ -181,13 +184,14 @@ public void testRestartAutomatically() throws Exception { sender.setInvokableClass(Tasks.NoOpInvokable.class); sender.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("Pointwise job", sender); + JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "Test job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000)); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -216,6 +220,7 @@ public void testCancelWhileRestarting() throws Exception { new JobID(), "TestJob", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), // We want to manually control the restart and delay new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE)); @@ -224,7 +229,7 @@ public void testCancelWhileRestarting() throws Exception { jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); jobVertex.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("TestJob", jobVertex); + JobGraph jobGraph = new JobGraph("TestJob", new ExecutionConfig(), jobVertex); executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -274,6 +279,7 @@ public void testCancelWhileFailing() throws Exception { new JobID(), "TestJob", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), // We want to manually control the restart and delay new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE)); @@ -289,7 +295,7 @@ public void testCancelWhileFailing() throws Exception { jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); jobVertex.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("TestJob", jobVertex); + JobGraph jobGraph = new JobGraph("TestJob", new ExecutionConfig(), jobVertex); executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -349,13 +355,14 @@ public void testNoRestartOnUnrecoverableException() throws Exception { sender.setInvokableClass(Tasks.NoOpInvokable.class); sender.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("Pointwise job", sender); + JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender); ExecutionGraph eg = spy(new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "Test job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000))); @@ -419,13 +426,14 @@ public void testFailingExecutionAfterRestart() throws Exception { receiver.setInvokableClass(Tasks.NoOpInvokable.class); receiver.setParallelism(1); - JobGraph jobGraph = new JobGraph("Pointwise job", sender, receiver); + JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000)); @@ -510,8 +518,8 @@ public void testFailExecutionAfterCancel() throws Exception { vertex.setInvokableClass(Tasks.NoOpInvokable.class); vertex.setParallelism(1); - JobGraph jobGraph = new JobGraph("Test Job", vertex); - jobGraph.setRestartStrategyConfiguration(RestartStrategies.fixedDelayRestart( + JobGraph jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex); + jobGraph.getExecutionConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart( Integer.MAX_VALUE, Integer.MAX_VALUE)); ExecutionGraph eg = new ExecutionGraph( @@ -519,6 +527,7 @@ public void testFailExecutionAfterCancel() throws Exception { new JobID(), "test job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000000)); @@ -561,8 +570,8 @@ public void testFailExecutionGraphAfterCancel() throws Exception { vertex.setInvokableClass(Tasks.NoOpInvokable.class); vertex.setParallelism(1); - JobGraph jobGraph = new JobGraph("Test Job", vertex); - jobGraph.setRestartStrategyConfiguration(RestartStrategies.fixedDelayRestart( + JobGraph jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex); + jobGraph.getExecutionConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart( Integer.MAX_VALUE, Integer.MAX_VALUE)); ExecutionGraph eg = new ExecutionGraph( @@ -570,6 +579,7 @@ public void testFailExecutionGraphAfterCancel() throws Exception { new JobID(), "test job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000000)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java index 7cc91c0d7b613..d1bb6801ee7b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.StoppingException; @@ -127,6 +128,7 @@ public void prepare() throws Exception { jobId, jobName, cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.attachJobGraph(ordered); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index ca07fbfb65b30..9519f0b1197a1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -25,6 +25,7 @@ import java.lang.reflect.Field; import java.net.InetAddress; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.akka.AkkaUtils; @@ -173,7 +174,8 @@ public static ExecutionJobVertex getExecutionVertex(JobVertexID id, ExecutionCon executionContext, new JobID(), "test job", - new Configuration(), + new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java index 5dd5ba6706fb0..d839ffb4a08c7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java @@ -1,86 +1,88 @@ -/* - * 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.executiongraph; - -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*; -import static org.junit.Assert.*; -import static org.mockito.Mockito.mock; - -import java.util.Arrays; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.junit.Test; - -public class ExecutionStateProgressTest { - - @Test - public void testAccumulatedStateFinished() { - try { - final JobID jid = new JobID(); - final JobVertexID vid = new JobVertexID(); - - JobVertex ajv = new JobVertex("TestVertex", vid); - ajv.setParallelism(3); - ajv.setInvokableClass(mock(AbstractInvokable.class).getClass()); - - ExecutionGraph graph = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jid, - "test job", - new Configuration(), - AkkaUtils.getDefaultTimeout(), - new NoRestartStrategy()); - graph.attachJobGraph(Arrays.asList(ajv)); - - setGraphStatus(graph, JobStatus.RUNNING); - - ExecutionJobVertex ejv = graph.getJobVertex(vid); - - // mock resources and mock taskmanager - for (ExecutionVertex ee : ejv.getTaskVertices()) { - SimpleSlot slot = getInstance( - new SimpleActorGateway( - TestingUtils.defaultExecutionContext()) - ).allocateSimpleSlot(jid); - ee.deployToSlot(slot); - } - - // finish all - for (ExecutionVertex ee : ejv.getTaskVertices()) { - ee.executionFinished(); - } - - assertTrue(ejv.isInFinalState()); - assertEquals(JobStatus.FINISHED, graph.getState()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } +/* + * 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.executiongraph; + +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*; +import static org.junit.Assert.*; +import static org.mockito.Mockito.mock; + +import java.util.Arrays; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; +import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.junit.Test; + +public class ExecutionStateProgressTest { + + @Test + public void testAccumulatedStateFinished() { + try { + final JobID jid = new JobID(); + final JobVertexID vid = new JobVertexID(); + + JobVertex ajv = new JobVertex("TestVertex", vid); + ajv.setParallelism(3); + ajv.setInvokableClass(mock(AbstractInvokable.class).getClass()); + + ExecutionGraph graph = new ExecutionGraph( + TestingUtils.defaultExecutionContext(), + jid, + "test job", + new Configuration(), + new ExecutionConfig(), + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); + graph.attachJobGraph(Arrays.asList(ajv)); + + setGraphStatus(graph, JobStatus.RUNNING); + + ExecutionJobVertex ejv = graph.getJobVertex(vid); + + // mock resources and mock taskmanager + for (ExecutionVertex ee : ejv.getTaskVertices()) { + SimpleSlot slot = getInstance( + new SimpleActorGateway( + TestingUtils.defaultExecutionContext()) + ).allocateSimpleSlot(jid); + ee.deployToSlot(slot); + } + + // finish all + for (ExecutionVertex ee : ejv.getTaskVertices()) { + ee.executionFinished(); + } + + assertTrue(ejv.isInFinalState()); + assertEquals(JobStatus.FINISHED, graph.getState()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } } \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java index a2e2482b76b41..2d7d6f53469e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java @@ -25,6 +25,7 @@ import java.net.InetAddress; import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.StrictlyLocalAssignment; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.core.io.InputSplitSource; @@ -264,13 +265,14 @@ public void testMultipleInstancesPerHost() { vertex.setInvokableClass(DummyInvokable.class); vertex.setInputSplitSource(new TestInputSplitSource(splits)); - JobGraph jobGraph = new JobGraph("test job", vertex); + JobGraph jobGraph = new JobGraph("test job", new ExecutionConfig(), vertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), + new ExecutionConfig(), TIMEOUT, new NoRestartStrategy()); @@ -328,13 +330,14 @@ private static String[] runTests(int numHosts, int slotsPerHost, int parallelism vertex.setInvokableClass(DummyInvokable.class); vertex.setInputSplitSource(new TestInputSplitSource(splits)); - JobGraph jobGraph = new JobGraph("test job", vertex); + JobGraph jobGraph = new JobGraph("test job", new ExecutionConfig(), vertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), + new ExecutionConfig(), TIMEOUT, new NoRestartStrategy()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java index c1afe04536a38..89879aae97891 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -63,6 +64,7 @@ public void testNToN() { jobId, jobName, cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -103,7 +105,8 @@ public void test2NToN() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -145,7 +148,8 @@ public void test3NToN() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -188,7 +192,8 @@ public void testNTo2N() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -229,7 +234,8 @@ public void testNTo7N() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -290,7 +296,8 @@ private void testLowToHigh(int lowDop, int highDop) { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -342,7 +349,8 @@ private void testHighToLow(int highDop, int lowDop) { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, + cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java index 2f0d5e707758e..f747ff362d75b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.executiongraph; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.execution.ExecutionState; @@ -185,6 +186,7 @@ static class TestExecGraph extends ExecutionGraph { jobId, "test graph", EMPTY_CONFIG, + new ExecutionConfig(), TIMEOUT, new FixedDelayRestartStrategy(1, 0)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java index 36703585a6009..25c9d70c9402b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; @@ -74,13 +75,14 @@ public void testScheduleWithConstraint1() { JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); jobVertex.setInvokableClass(DummyInvokable.class); jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", jobVertex); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), + new ExecutionConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); @@ -146,13 +148,14 @@ public void testScheduleWithConstraint2() { JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); jobVertex.setInvokableClass(DummyInvokable.class); jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", jobVertex); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), + new ExecutionConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); @@ -222,13 +225,14 @@ public void testScheduleWithConstraintAndSlotSharing() { jobVertex1.setSlotSharingGroup(sharingGroup); jobVertex2.setSlotSharingGroup(sharingGroup); - JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex1, jobVertex2); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), + new ExecutionConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); @@ -289,13 +293,14 @@ public void testScheduleWithUnfulfillableConstraint() { JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); jobVertex.setInvokableClass(DummyInvokable.class); jobVertex.setParallelism(1); - JobGraph jg = new JobGraph("test job", jobVertex); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), + new ExecutionConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); @@ -354,7 +359,7 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { jobVertex1.setParallelism(1); jobVertex2.setParallelism(1); - JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex1, jobVertex2); SlotSharingGroup sharingGroup = new SlotSharingGroup(); jobVertex1.setSlotSharingGroup(sharingGroup); @@ -365,6 +370,7 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), + new ExecutionConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); @@ -397,13 +403,14 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { public void testArchivingClearsFields() { try { JobVertex vertex = new JobVertex("test vertex", new JobVertexID()); - JobGraph jg = new JobGraph("test job", vertex); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), vertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), + new ExecutionConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(vertex)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java index 5c7297e5b3136..511024902851c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java @@ -24,6 +24,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; @@ -75,6 +76,7 @@ public void testAssignSlotSharingGroup() { new JobID(), "test job", new Configuration(), + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.attachJobGraph(vertices); 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 854be5f2a2aea..317eed7edf47d 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 @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.api.reader.BufferReader; @@ -89,7 +90,7 @@ public void testPartialConsumePipelinedResultReceiver() throws Exception { sender, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); final JobGraph jobGraph = new JobGraph( - "Partial Consume of Pipelined Result", sender, receiver); + "Partial Consume of Pipelined Result", new ExecutionConfig(), sender, receiver); final SlotSharingGroup slotSharingGroup = new SlotSharingGroup( sender.getID(), receiver.getID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java index ca047e8a60301..68b05b209a01c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java @@ -22,6 +22,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.core.testutils.CommonTestUtils; import org.junit.Test; @@ -31,7 +32,7 @@ public class JobGraphTest { @Test public void testSerialization() { try { - JobGraph jg = new JobGraph("The graph"); + JobGraph jg = new JobGraph("The graph", new ExecutionConfig()); // add some configuration values { @@ -89,8 +90,9 @@ public void testTopologicalSort1() { target2.connectNewDataSetAsInput(intermediate2, DistributionPattern.POINTWISE); intermediate2.connectNewDataSetAsInput(intermediate1, DistributionPattern.POINTWISE); intermediate1.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE); - - JobGraph graph = new JobGraph("TestGraph", source1, source2, intermediate1, intermediate2, target1, target2); + + JobGraph graph = new JobGraph("TestGraph", new ExecutionConfig(), + source1, source2, intermediate1, intermediate2, target1, target2); List sorted = graph.getVerticesSortedTopologicallyFromSources(); assertEquals(6, sorted.size()); @@ -133,8 +135,9 @@ public void testTopologicalSort2() { l12.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE); l13.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE); - - JobGraph graph = new JobGraph("TestGraph", source1, source2, root, l11, l13, l12, l2); + + JobGraph graph = new JobGraph("TestGraph", new ExecutionConfig(), + source1, source2, root, l11, l13, l12, l2); List sorted = graph.getVerticesSortedTopologicallyFromSources(); assertEquals(7, sorted.size()); @@ -179,8 +182,8 @@ public void testTopologicalSort3() { op2.connectNewDataSetAsInput(op1, DistributionPattern.POINTWISE); op2.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE); op3.connectNewDataSetAsInput(op2, DistributionPattern.POINTWISE); - - JobGraph graph = new JobGraph("TestGraph", source, op1, op2, op3); + + JobGraph graph = new JobGraph("TestGraph", new ExecutionConfig(), source, op1, op2, op3); List sorted = graph.getVerticesSortedTopologicallyFromSources(); assertEquals(4, sorted.size()); @@ -208,8 +211,8 @@ public void testTopoSortCyclicGraphNoSources() { v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE); v3.connectNewDataSetAsInput(v2, DistributionPattern.POINTWISE); v4.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE); - - JobGraph jg = new JobGraph("Cyclic Graph", v1, v2, v3, v4); + + JobGraph jg = new JobGraph("Cyclic Graph", new ExecutionConfig(), v1, v2, v3, v4); try { jg.getVerticesSortedTopologicallyFromSources(); fail("Failed to raise error on topologically sorting cyclic graph."); @@ -240,8 +243,8 @@ public void testTopoSortCyclicGraphIntermediateCycle() { v3.connectNewDataSetAsInput(v2, DistributionPattern.POINTWISE); v4.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE); target.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE); - - JobGraph jg = new JobGraph("Cyclic Graph", v1, v2, v3, v4, source, target); + + JobGraph jg = new JobGraph("Cyclic Graph", new ExecutionConfig(), v1, v2, v3, v4, source, target); try { jg.getVerticesSortedTopologicallyFromSources(); fail("Failed to raise error on topologically sorting cyclic graph."); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java index 46fb694545a94..612f64f9c9f25 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.TextNode; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -65,8 +66,8 @@ public void testGeneratorWithoutAnyAttachements() { sink1.connectNewDataSetAsInput(join2, DistributionPattern.POINTWISE); sink2.connectNewDataSetAsInput(join1, DistributionPattern.ALL_TO_ALL); - - JobGraph jg = new JobGraph("my job", source1, source2, source3, + + JobGraph jg = new JobGraph("my job", new ExecutionConfig(), source1, source2, source3, intermediate1, intermediate2, join1, join2, sink1, sink2); String plan = JsonPlanGenerator.generatePlan(jg); 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 27c32a05edf4a..d283e91713f33 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 @@ -23,6 +23,7 @@ import com.typesafe.config.Config; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -122,7 +123,7 @@ public void testRequestPartitionState() throws Exception { sender.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block sender.createAndAddResultDataSet(rid, PIPELINED); - final JobGraph jobGraph = new JobGraph("Blocking test job", sender); + final JobGraph jobGraph = new JobGraph("Blocking test job", new ExecutionConfig(), sender); final JobID jid = jobGraph.getJobID(); final ActorGateway jobManagerGateway = cluster.getLeaderGateway( @@ -246,7 +247,7 @@ public void testStopSignal() throws Exception { sender.setParallelism(2); sender.setInvokableClass(StoppableInvokable.class); - final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", sender); + final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", new ExecutionConfig(), sender); final JobID jid = jobGraph.getJobID(); final ActorGateway jobManagerGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION()); @@ -294,7 +295,7 @@ public void testStopSignalFail() throws Exception { sender.setParallelism(1); sender.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block - final JobGraph jobGraph = new JobGraph("Non-Stoppable batching test job", sender); + final JobGraph jobGraph = new JobGraph("Non-Stoppable batching test job", new ExecutionConfig(), sender); final JobID jid = jobGraph.getJobID(); final ActorGateway jobManagerGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java index adaff29040dc5..1cd01ffb09411 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java @@ -20,6 +20,7 @@ import akka.actor.ActorRef; import akka.actor.ActorSystem; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -105,7 +106,7 @@ public void testFailureWhenJarBlobsMissing() { // create a simple job graph JobVertex jobVertex = new JobVertex("Test Vertex"); jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - JobGraph jg = new JobGraph("test job", jobVertex); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); // request the blob port from the job manager Future future = jmGateway.ask(JobManagerMessages.getRequestBlobManagerPort(), timeout); @@ -169,7 +170,7 @@ public void initializeOnMaster(ClassLoader loader) throws Exception { }; jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - JobGraph jg = new JobGraph("test job", jobVertex); + JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); // submit the job Future submitFuture = jmGateway.ask( 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 17334064605a2..dfb0b91d11eec 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 @@ -18,6 +18,7 @@ package org.apache.flink.runtime.jobmanager; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.io.network.api.reader.RecordReader; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; @@ -103,7 +104,7 @@ private JobGraph createTestJobGraph( DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - final JobGraph jobGraph = new JobGraph(jobName, sender, receiver); + final JobGraph jobGraph = new JobGraph(jobName, new ExecutionConfig(), sender, receiver); // We need to allow queued scheduling, because there are not enough slots available // to run all tasks at once. We queue tasks and then let them finish/consume the blocking diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java index 753e7be2a0080..ca2ecf515ec54 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.jobmanager; import akka.actor.ActorRef; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -37,7 +38,7 @@ public void testNoOps() throws Exception { StandaloneSubmittedJobGraphStore jobGraphs = new StandaloneSubmittedJobGraphStore(); SubmittedJobGraph jobGraph = new SubmittedJobGraph( - new JobGraph("testNoOps"), + new JobGraph("testNoOps", new ExecutionConfig()), new JobInfo(ActorRef.noSender(), ListeningBehaviour.DETACHED, 0, Integer.MAX_VALUE)); assertEquals(0, jobGraphs.recoverJobGraphs().size()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java index 356ba3687ac81..5e53596fab28f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.jobmanager; import akka.actor.ActorRef; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -260,7 +261,7 @@ public void testUpdateJobGraphYouDidNotGetOrAdd() throws Exception { // --------------------------------------------------------------------------------------------- private SubmittedJobGraph createSubmittedJobGraph(JobID jobId, long start) { - final JobGraph jobGraph = new JobGraph(jobId, "Test JobGraph"); + final JobGraph jobGraph = new JobGraph(jobId, "Test JobGraph", new ExecutionConfig()); final JobVertex jobVertex = new JobVertex("Test JobVertex"); jobVertex.setParallelism(1); 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 eb4d96fb29759..07fc2c550e1ed 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 @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; +import org.apache.flink.api.common.ExecutionConfig; 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; @@ -112,6 +113,7 @@ public void testMixedPipelinedAndBlockingResults() throws Exception { final JobGraph jobGraph = new JobGraph( "Mixed pipelined and blocking result", + new ExecutionConfig(), sender, pipelinedReceiver, blockingReceiver); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java index c490a640c3269..f14d62f34bc4e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.leaderelection; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.instance.ActorGateway; @@ -268,6 +269,6 @@ public JobGraph createBlockingJob(int parallelism) { sender.setSlotSharingGroup(slotSharingGroup); receiver.setSlotSharingGroup(slotSharingGroup); - return new JobGraph("Blocking test job", sender, receiver); + return new JobGraph("Blocking test job", new ExecutionConfig(), sender, receiver); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java index 3fcc42539c467..233dace253e9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.concurrent.Future; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; @@ -44,11 +45,17 @@ public class DummyEnvironment implements Environment { private final TaskInfo taskInfo; private final JobID jobId = new JobID(); private final JobVertexID jobVertexId = new JobVertexID(); + private final ExecutionConfig executionConfig = new ExecutionConfig(); public DummyEnvironment(String taskName, int numSubTasks, int subTaskIndex) { this.taskInfo = new TaskInfo(taskName, subTaskIndex, numSubTasks, 0); } + @Override + public ExecutionConfig getExecutionConfig() { + return executionConfig; + } + @Override public JobID getJobID() { return jobId; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index fa97210c98347..9ee5bd222b512 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators.testutils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.UnmodifiableConfiguration; @@ -64,7 +65,9 @@ public class MockEnvironment implements Environment { private final TaskInfo taskInfo; - + + private final ExecutionConfig executionConfig; + private final MemoryManager memManager; private final IOManager ioManager; @@ -96,6 +99,7 @@ public MockEnvironment(String taskName, long memorySize, MockInputSplitProvider this.memManager = new MemoryManager(memorySize, 1); this.ioManager = new IOManagerAsync(); + this.executionConfig = new ExecutionConfig(); this.inputSplitProvider = inputSplitProvider; this.bufferSize = bufferSize; @@ -185,6 +189,11 @@ public IOManager getIOManager() { return this.ioManager; } + @Override + public ExecutionConfig getExecutionConfig() { + return this.executionConfig; + } + @Override public JobID getJobID() { return this.jobID; 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 e5ff7b1654ae2..9d33920b96ba4 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 @@ -18,6 +18,7 @@ package org.apache.flink.runtime.taskmanager; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; @@ -148,7 +149,7 @@ private static Task createTask() { TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), new JobVertexID(), new ExecutionAttemptID(), - "Test Task", 0, 1, 0, + new ExecutionConfig(), "Test Task", 0, 1, 0, new Configuration(), new Configuration(), CheckpointsInOrderInvokable.class.getName(), Collections.emptyList(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java index a3d18831c4989..e7f4c5c4a2bee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.taskmanager; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -73,7 +74,7 @@ public void testCancelUnion() throws Exception { flink.start(); // Setup - final JobGraph jobGraph = new JobGraph("Cancel Big Union"); + final JobGraph jobGraph = new JobGraph("Cancel Big Union", new ExecutionConfig()); JobVertex[] sources = new JobVertex[numberOfSources]; SlotSharingGroup group = new SlotSharingGroup(); 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 b9f4cbf68939c..86565d185e862 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 @@ -23,6 +23,7 @@ import akka.actor.Props; import akka.japi.Creator; import akka.testkit.JavaTestKit; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -157,9 +158,11 @@ protected void run() { final JobID jid = new JobID(); final JobVertexID vid = new JobVertexID(); final ExecutionAttemptID eid = new ExecutionAttemptID(); + final ExecutionConfig executionConfig = new ExecutionConfig(); - final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, "TestTask", 2, 7, 0, - new Configuration(), new Configuration(), TestInvokableCorrect.class.getName(), + final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, executionConfig, + "TestTask", 2, 7, 0, new Configuration(), new Configuration(), + TestInvokableCorrect.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); @@ -258,13 +261,15 @@ public void testJobSubmissionAndCanceling() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, "TestTask1", 1, 5, 0, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, + new ExecutionConfig(), "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, "TestTask2", 2, 7, 0, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, + new ExecutionConfig(), "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), @@ -390,14 +395,14 @@ public void testJobSubmissionAndStop() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, "TestTask1", 1, 5, 0, - new Configuration(), new Configuration(), StoppableInvokable.class.getName(), + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, new ExecutionConfig(), + "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), StoppableInvokable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, "TestTask2", 2, 7, 0, - new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, new ExecutionConfig(), + "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); @@ -516,13 +521,15 @@ public void testGateChannelEdgeMismatch() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1, 0, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + new ExecutionConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7, 0, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + new ExecutionConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), Collections.emptyList(), @@ -615,12 +622,14 @@ public void testRunJobWithForwardChannel() { } ); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1, 0, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + new ExecutionConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7, 0, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + new ExecutionConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), Collections.singletonList(ircdd), @@ -754,12 +763,14 @@ public void testCancellingDependentAndStateUpdateFails() { } ); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1, 0, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + new ExecutionConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7, 0, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + new ExecutionConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.BlockingReceiver.class.getName(), Collections.emptyList(), Collections.singletonList(ircdd), @@ -897,7 +908,8 @@ public void testRemotePartitionNotFound() throws Exception { new InputGateDeploymentDescriptor(resultId, 0, icdd); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - jid, vid, eid, "Receiver", 0, 1, 0, + jid, vid, eid, + new ExecutionConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), Collections.emptyList(), @@ -990,7 +1002,7 @@ public void testLocalPartitionNotFound() throws Exception { new InputGateDeploymentDescriptor(resultId, 0, icdd); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - jid, vid, eid, "Receiver", 0, 1, 0, + jid, vid, eid, new ExecutionConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), Collections.emptyList(), @@ -1066,6 +1078,7 @@ public void testTriggerStackTraceSampleMessage() throws Exception { new JobID(), new JobVertexID(), new ExecutionAttemptID(), + new ExecutionConfig(), "Task", 0, 1, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index a60e0746f95b9..aa37d4769487d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -17,6 +17,7 @@ */ package org.apache.flink.runtime.taskmanager; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; @@ -61,6 +62,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { when(tddMock.getExecutionId()).thenReturn(mock(ExecutionAttemptID.class)); when(tddMock.getJobConfiguration()).thenReturn(mock(Configuration.class)); when(tddMock.getTaskConfiguration()).thenReturn(mock(Configuration.class)); + when(tddMock.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); when(tddMock.getInvokableClassName()).thenReturn("className"); task = new Task(tddMock, mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), 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 45ca364b7e77e..034681e5c32c3 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 @@ -20,6 +20,7 @@ import com.google.common.collect.Maps; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -627,7 +628,7 @@ private Task createTask(Class invokable, private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class invokable) { return new TaskDeploymentDescriptor( new JobID(), new JobVertexID(), new ExecutionAttemptID(), - "Test Task", 0, 1, 0, + new ExecutionConfig(), "Test Task", 0, 1, 0, new Configuration(), new Configuration(), invokable.getName(), Collections.emptyList(), diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala index dbd87d0262421..ef3dae42203c9 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph -import org.apache.flink.api.common.JobID +import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleActorGateway @@ -50,13 +50,14 @@ class TaskManagerLossFailsTasksTest extends WordSpecLike with Matchers { sender.setInvokableClass(classOf[Tasks.NoOpInvokable]) sender.setParallelism(20) - val jobGraph = new JobGraph("Pointwise job", sender) + val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender) val eg = new ExecutionGraph( TestingUtils.defaultExecutionContext, new JobID(), "test job", new Configuration(), + new ExecutionConfig, AkkaUtils.getDefaultTimeout, new NoRestartStrategy()) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala index a0c144a785eb3..f52d37e5ee854 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala @@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobmanager import akka.actor.ActorSystem import akka.actor.Status.Success import akka.testkit.{ImplicitSender, TestKit} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.runtime.akka.ListeningBehaviour import org.apache.flink.runtime.jobgraph.{JobGraph, DistributionPattern, JobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{Receiver, Sender} @@ -67,7 +68,7 @@ class CoLocationConstraintITCase(_system: ActorSystem) receiver.setStrictlyCoLocatedWith(sender) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala index ec54b7e3f4f0c..bf8df5ae96f60 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala @@ -22,9 +22,9 @@ package org.apache.flink.runtime.jobmanager import akka.actor.ActorSystem import akka.testkit.{ImplicitSender, TestKit} import akka.util.Timeout -import org.apache.flink.api.common.JobID +import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.runtime.akka.ListeningBehaviour -import org.apache.flink.runtime.checkpoint.{CheckpointCoordinator, SavepointCoordinator} +import org.apache.flink.runtime.checkpoint.SavepointCoordinator import org.apache.flink.runtime.client.JobExecutionException import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings import org.apache.flink.runtime.jobgraph.{DistributionPattern, JobGraph, JobVertex, ScheduleMode} @@ -33,7 +33,6 @@ import org.apache.flink.runtime.jobmanager.scheduler.{NoResourceAvailableExcepti import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils} -import org.apache.flink.runtime.testutils.JobManagerActorTestUtils import org.junit.runner.RunWith import org.mockito.Mockito._ import org.scalatest.junit.JUnitRunner @@ -68,7 +67,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(2) vertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph("Test Job", vertex) + val jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex) val cluster = TestingUtils.startTestingCluster(1) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -110,7 +109,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(num_tasks) vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph = new JobGraph("Test Job", vertex) + val jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -145,7 +144,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(num_tasks) vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph = new JobGraph("Test job", vertex) + val jobGraph = new JobGraph("Test job", new ExecutionConfig(), vertex) jobGraph.setAllowQueuedScheduling(true) val cluster = TestingUtils.startTestingCluster(10) @@ -181,7 +180,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(2 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -216,7 +215,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Bipartite Job", sender, receiver) + val jobGraph = new JobGraph("Bipartite Job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(2 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -253,7 +252,8 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) receiver.connectNewDataSetAsInput(sender2, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) + val jobGraph = new JobGraph("Bipartite Job", new ExecutionConfig(), + sender1, receiver, sender2) val cluster = TestingUtils.startTestingCluster(6 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -298,7 +298,8 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) receiver.connectNewDataSetAsInput(sender2, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) + val jobGraph = new JobGraph("Bipartite Job", new ExecutionConfig(), + sender1, receiver, sender2) val cluster = TestingUtils.startTestingCluster(6 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -340,7 +341,8 @@ class JobManagerITCase(_system: ActorSystem) forwarder.connectNewDataSetAsInput(sender, DistributionPattern.ALL_TO_ALL) receiver.connectNewDataSetAsInput(forwarder, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Forwarding Job", sender, forwarder, receiver) + val jobGraph = new JobGraph("Forwarding Job", new ExecutionConfig(), + sender, forwarder, receiver) jobGraph.setScheduleMode(ScheduleMode.ALL) @@ -376,7 +378,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -424,7 +426,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -469,7 +471,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(2 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -509,7 +511,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -557,7 +559,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -600,7 +602,8 @@ class JobManagerITCase(_system: ActorSystem) source.setParallelism(num_tasks) sink.setParallelism(num_tasks) - val jobGraph = new JobGraph("SubtaskInFinalStateRaceCondition", source, sink) + val jobGraph = new JobGraph("SubtaskInFinalStateRaceCondition", + new ExecutionConfig(), source, sink) val cluster = TestingUtils.startTestingCluster(2*num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -626,12 +629,12 @@ class JobManagerITCase(_system: ActorSystem) val vertex = new JobVertex("Test Vertex") vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph1 = new JobGraph("Test Job", vertex) + val jobGraph1 = new JobGraph("Test Job", new ExecutionConfig(), vertex) val slowVertex = new WaitingOnFinalizeJobVertex("Long running Vertex", 2000) slowVertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph2 = new JobGraph("Long running Job", slowVertex) + val jobGraph2 = new JobGraph("Long running Job", new ExecutionConfig(), slowVertex) val cluster = TestingUtils.startTestingCluster(1) val jm = cluster.getLeaderGateway(1 seconds) @@ -680,7 +683,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(1) vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph = new JobGraph("Test Job", vertex) + val jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex) val cluster = TestingUtils.startTestingCluster(1) val jm = cluster.getLeaderGateway(1 seconds) @@ -778,7 +781,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(jobVertex) + val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) // Submit job w/o checkpointing configured jobManager.tell(SubmitJob(jobGraph, ListeningBehaviour.DETACHED), testActor) @@ -811,7 +814,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(jobVertex) + val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) jobGraph.setSnapshotSettings(new JobSnapshottingSettings( java.util.Collections.emptyList(), java.util.Collections.emptyList(), @@ -864,7 +867,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(jobVertex) + val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) jobGraph.setSnapshotSettings(new JobSnapshottingSettings( java.util.Collections.emptyList(), java.util.Collections.emptyList(), @@ -922,7 +925,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(jobVertex) + val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) jobGraph.setSnapshotSettings(new JobSnapshottingSettings( java.util.Collections.emptyList(), java.util.Collections.emptyList(), diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala index 41b6702d71437..ea42cd14c7aa5 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala @@ -20,6 +20,7 @@ package org.apache.flink.runtime.jobmanager import akka.actor.{PoisonPill, ActorSystem} import akka.testkit.{ImplicitSender, TestKit} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.configuration.{ConfigConstants, Configuration} import org.apache.flink.runtime.akka.ListeningBehaviour import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, DistributionPattern, JobVertex} @@ -82,7 +83,10 @@ class RecoveryITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val executionConfig = new ExecutionConfig() + executionConfig.setNumberOfExecutionRetries(1); + + val jobGraph = new JobGraph("Pointwise job", executionConfig, sender, receiver) val cluster = createTestClusterWithHeartbeatTimeout(2 * NUM_TASKS, 1, "2 s") cluster.start() @@ -126,7 +130,10 @@ class RecoveryITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val executionConfig = new ExecutionConfig() + executionConfig.setNumberOfExecutionRetries(1); + + val jobGraph = new JobGraph("Pointwise job", executionConfig, sender, receiver) val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 1, "2 s") cluster.start() @@ -170,7 +177,10 @@ class RecoveryITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise job", sender, receiver) + val executionConfig = new ExecutionConfig() + executionConfig.setNumberOfExecutionRetries(1); + + val jobGraph = new JobGraph("Pointwise job", executionConfig, sender, receiver) val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 2, "2 s") cluster.start() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala index a6d60dd2f45da..4dc80c383f796 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala @@ -19,8 +19,8 @@ package org.apache.flink.runtime.jobmanager import akka.actor.ActorSystem -import akka.actor.Status.Success import akka.testkit.{ImplicitSender, TestKit} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.runtime.akka.ListeningBehaviour import org.apache.flink.runtime.jobgraph.{JobVertex, DistributionPattern, JobGraph} import org.apache.flink.runtime.jobmanager.Tasks.{Sender, AgnosticBinaryReceiver, Receiver} @@ -65,7 +65,7 @@ class SlotSharingITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -109,7 +109,8 @@ class SlotSharingITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) receiver.connectNewDataSetAsInput(sender2, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Bipartite job", sender1, sender2, receiver) + val jobGraph = new JobGraph("Bipartite job", new ExecutionConfig(), + sender1, sender2, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala index 49a1c9535caa4..c108596ef2280 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala @@ -20,6 +20,7 @@ package org.apache.flink.runtime.jobmanager import akka.actor.{Kill, ActorSystem, PoisonPill} import akka.testkit.{ImplicitSender, TestKit} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.runtime.akka.ListeningBehaviour import org.apache.flink.runtime.client.JobExecutionException import org.apache.flink.runtime.jobgraph.{JobVertex, DistributionPattern, JobGraph} @@ -28,7 +29,6 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup import org.apache.flink.runtime.messages.JobManagerMessages.{JobResultFailure, JobSubmitSuccess, SubmitJob} import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils} -import org.apache.flink.runtime.util.SerializedThrowable import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike} @@ -67,7 +67,7 @@ class TaskManagerFailsWithSlotSharingITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val jobID = jobGraph.getJobID val cluster = TestingUtils.startTestingCluster(num_tasks/2, 2) @@ -116,7 +116,7 @@ class TaskManagerFailsWithSlotSharingITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val jobID = jobGraph.getJobID val cluster = TestingUtils.startTestingCluster(num_tasks/2, 2) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index e3e1ac612dff1..1cb8f83d51662 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -24,7 +24,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; @@ -52,12 +51,10 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.tasks.StreamIterationHead; import org.apache.flink.streaming.runtime.tasks.StreamIterationTail; -import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.nio.charset.Charset; import java.util.ArrayDeque; import java.util.ArrayList; @@ -112,7 +109,7 @@ private void init() { } public JobGraph createJobGraph() { - jobGraph = new JobGraph(streamGraph.getJobName()); + jobGraph = new JobGraph(streamGraph.getJobName(), streamGraph.getExecutionConfig()); // make sure that all vertices start immediately jobGraph.setScheduleMode(ScheduleMode.ALL); @@ -133,12 +130,6 @@ public JobGraph createJobGraph() { configureRestartStrategy(); - try { - InstantiationUtil.writeObjectToConfig(this.streamGraph.getExecutionConfig(), this.jobGraph.getJobConfiguration(), ExecutionConfig.CONFIG_KEY); - } catch (IOException e) { - throw new RuntimeException("Config object could not be written to Job Configuration: ", e); - } - return jobGraph; } @@ -327,7 +318,7 @@ private void setVertexConfig(Integer vertexID, StreamConfig config, config.setChainedOutputs(chainableOutputs); config.setTimeCharacteristic(streamGraph.getEnvironment().getStreamTimeCharacteristic()); - + final CheckpointConfig ceckpointCfg = streamGraph.getCheckpointConfig(); config.setStateBackend(streamGraph.getStateBackend()); @@ -494,7 +485,8 @@ private void configureCheckpointing() { } private void configureRestartStrategy() { - jobGraph.setRestartStrategyConfiguration(streamGraph.getExecutionConfig().getRestartStrategy()); + jobGraph.getExecutionConfig().setRestartStrategy( + streamGraph.getExecutionConfig().getRestartStrategy()); } // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java index 9f75727f0cd14..05175764f894a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java @@ -42,7 +42,7 @@ public void testAutomaticRestartingWhenCheckpointing() { StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); - RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getRestartStrategyConfiguration(); + RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getExecutionConfig().getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.FixedDelayRestartStrategyConfiguration); @@ -64,7 +64,7 @@ public void testNoRestartingWhenCheckpointingAndExplicitExecutionRetriesZero() { StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); - RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getRestartStrategyConfiguration(); + RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getExecutionConfig().getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.NoRestartStrategyConfiguration); @@ -86,7 +86,7 @@ public void testFixedRestartingWhenCheckpointingAndExplicitExecutionRetriesNonZe StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); - RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getRestartStrategyConfiguration(); + RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getExecutionConfig().getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.FixedDelayRestartStrategyConfiguration); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java index 8a814ff4d9462..b89bd75d376c3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java @@ -44,10 +44,10 @@ public void testExecutionConfigSerialization() throws IOException, ClassNotFound StreamGraph streamingJob = new StreamGraph(env); StreamingJobGraphGenerator compiler = new StreamingJobGraphGenerator(streamingJob); - + boolean closureCleanerEnabled = r.nextBoolean(), forceAvroEnabled = r.nextBoolean(), forceKryoEnabled = r.nextBoolean(), objectReuseEnabled = r.nextBoolean(), sysoutLoggingEnabled = r.nextBoolean(); int dop = 1 + r.nextInt(10); - + ExecutionConfig config = streamingJob.getExecutionConfig(); if(closureCleanerEnabled) { config.enableClosureCleaner(); @@ -75,16 +75,22 @@ public void testExecutionConfigSerialization() throws IOException, ClassNotFound config.disableSysoutLogging(); } config.setParallelism(dop); - + JobGraph jobGraph = compiler.createJobGraph(); - + + final String EXEC_CONFIG_KEY = "runtime.config"; + + InstantiationUtil.writeObjectToConfig(jobGraph.getExecutionConfig(), + jobGraph.getJobConfiguration(), + EXEC_CONFIG_KEY); + ExecutionConfig executionConfig = InstantiationUtil.readObjectFromConfig( jobGraph.getJobConfiguration(), - ExecutionConfig.CONFIG_KEY, + EXEC_CONFIG_KEY, Thread.currentThread().getContextClassLoader()); - + assertNotNull(executionConfig); - + assertEquals(closureCleanerEnabled, executionConfig.isClosureCleanerEnabled()); assertEquals(forceAvroEnabled, executionConfig.isForceAvroEnabled()); assertEquals(forceKryoEnabled, executionConfig.isForceKryoEnabled()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java index 184f87e6ef39c..732d3e5404f80 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.runtime.partitioner; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple; @@ -134,6 +135,7 @@ public void flatMap(String value, jobId, jobName, cfg, + new ExecutionConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy(), new ArrayList(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index c4b74e8921ee8..f91353e65a9e4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.tasks; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -90,8 +91,10 @@ public class StreamMockEnvironment implements Environment { private final int bufferSize; - public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, long memorySize, - MockInputSplitProvider inputSplitProvider, int bufferSize) { + private final ExecutionConfig executionConfig; + + public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, ExecutionConfig executionConfig, + long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) { this.taskInfo = new TaskInfo("", 0, 1, 0); this.jobConfiguration = jobConfig; this.taskConfiguration = taskConfig; @@ -103,9 +106,15 @@ public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, this.inputSplitProvider = inputSplitProvider; this.bufferSize = bufferSize; + this.executionConfig = executionConfig; this.accumulatorRegistry = new AccumulatorRegistry(jobID, getExecutionId()); } + public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, long memorySize, + MockInputSplitProvider inputSplitProvider, int bufferSize) { + this(jobConfig, taskConfig, null, memorySize, inputSplitProvider, bufferSize); + } + public void addInputGate(InputGate gate) { inputs.add(gate); } @@ -205,6 +214,11 @@ public IOManager getIOManager() { return this.ioManager; } + @Override + public ExecutionConfig getExecutionConfig() { + return this.executionConfig; + } + @Override public JobID getJobID() { return this.jobID; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java index 1830054cfa4d1..ed1dd60cb0858 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.tasks; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; @@ -76,6 +77,11 @@ public void testAsyncCheckpoints() throws Exception { new MockInputSplitProvider(), testHarness.bufferSize) { + @Override + public ExecutionConfig getExecutionConfig() { + return testHarness.executionConfig; + } + @Override public void acknowledgeCheckpoint(long checkpointId) { super.acknowledgeCheckpoint(checkpointId); 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 7cc58b548b41e..7f4492a74dbde 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 @@ -20,6 +20,7 @@ import akka.actor.ActorRef; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; @@ -134,7 +135,7 @@ private Task createTask(Class invokable, StreamConf TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), new JobVertexID(), new ExecutionAttemptID(), - "Test Task", 0, 1, 0, + new ExecutionConfig(), "Test Task", 0, 1, 0, new Configuration(), taskConfig.getConfiguration(), invokable.getName(), 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 8dc7edde4029b..e750f6fd3e05e 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 @@ -37,7 +37,6 @@ import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; -import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; import java.io.IOException; @@ -104,12 +103,6 @@ public StreamTaskTestHarness(AbstractInvokable task, TypeInformation output this.jobConfig = new Configuration(); this.taskConfig = new Configuration(); this.executionConfig = new ExecutionConfig(); - - try { - InstantiationUtil.writeObjectToConfig(executionConfig, this.jobConfig, ExecutionConfig.CONFIG_KEY); - } catch (IOException e) { - throw new RuntimeException(e); - } streamConfig = new StreamConfig(taskConfig); streamConfig.setChainStart(); @@ -156,7 +149,8 @@ private void initializeOutput() { * Task thread to finish running. */ public void invoke() throws Exception { - mockEnv = new StreamMockEnvironment(jobConfig, taskConfig, memorySize, new MockInputSplitProvider(), bufferSize); + mockEnv = new StreamMockEnvironment(jobConfig, taskConfig, executionConfig, + memorySize, new MockInputSplitProvider(), bufferSize); task.setEnvironment(mockEnv); initializeInputs(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java index 28c2e5837fc95..f6c22d439f958 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.failingPrograms; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -64,7 +65,7 @@ public static void setup() { final JobVertex jobVertex = new JobVertex("Working job vertex."); jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - workingJobGraph = new JobGraph("Working testing job", jobVertex); + workingJobGraph = new JobGraph("Working testing job", new ExecutionConfig(), jobVertex); } catch (Exception e) { e.printStackTrace(); @@ -115,7 +116,7 @@ public void testExceptionInInitializeOnMaster() { final JobVertex failingJobVertex = new FailingJobVertex("Failing job vertex"); failingJobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - final JobGraph failingJobGraph = new JobGraph("Failing testing job", failingJobVertex); + final JobGraph failingJobGraph = new JobGraph("Failing testing job", new ExecutionConfig(), failingJobVertex); try { submitJob(failingJobGraph); @@ -140,7 +141,7 @@ public void testExceptionInInitializeOnMaster() { @Test public void testSubmitEmptyJobGraph() { try { - final JobGraph jobGraph = new JobGraph("Testing job"); + final JobGraph jobGraph = new JobGraph("Testing job", new ExecutionConfig()); try { submitJob(jobGraph); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java index e5a494b2f0272..c1bd5e2fdd8ef 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java @@ -68,6 +68,39 @@ public void testIdentityMapWithBasicType() throws Exception { compareResultAsText(result, expected); } + @Test + public void testRuntimeContextAndExecutionConfigParams() throws Exception { + /* + * Test identity map with basic type + */ + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().setNumberOfExecutionRetries(1000); + + DataSet ds = CollectionDataSets.getStringDataSet(env); + DataSet identityMapDs = ds. + map(new RichMapFunction() { + @Override + public String map(String value) throws Exception { + Assert.assertTrue(1000 == getRuntimeContext().getExecutionConfig().getNumberOfExecutionRetries()); + return value; + } + }); + + List result = identityMapDs.collect(); + + String expected = "Hi\n" + + "Hello\n" + + "Hello world\n" + + "Hello world, how are you?\n" + + "I am fine.\n" + + "Luke Skywalker\n" + + "Random comment\n" + + "LOL\n"; + + compareResultAsText(result, expected); + } + public static class Mapper1 implements MapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java index 5783fccd3451a..e8ad5277ef23e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java @@ -21,6 +21,7 @@ import akka.actor.ActorRef; import akka.actor.ActorSystem; import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -366,7 +367,7 @@ public void testCheckpointRecoveryFailure() throws Exception { // BLocking JobGraph JobVertex blockingVertex = new JobVertex("Blocking vertex"); blockingVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class); - JobGraph jobGraph = new JobGraph(blockingVertex); + JobGraph jobGraph = new JobGraph(new ExecutionConfig(), blockingVertex); // Submit the job in detached mode leader.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java index 5080aa2a1b80a..32423bea9f333 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java @@ -25,6 +25,7 @@ import akka.testkit.TestActorRef; import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -428,7 +429,7 @@ public void awaitJobResult(long timeout) throws InterruptedException { * Creates a simple blocking JobGraph. */ private static JobGraph createBlockingJobGraph() { - JobGraph jobGraph = new JobGraph("Blocking program"); + JobGraph jobGraph = new JobGraph("Blocking program", new ExecutionConfig()); JobVertex jobVertex = new JobVertex("Blocking Vertex"); jobVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class); 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 aada3647ced63..a87057845b7da 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 @@ -18,6 +18,7 @@ package org.apache.flink.test.runtime; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; @@ -96,7 +97,7 @@ protected JobGraph getJobGraph() throws Exception { private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender, boolean isSlowReceiver, int numSubtasks) { - JobGraph jobGraph = new JobGraph("Speed Test"); + JobGraph jobGraph = new JobGraph("Speed Test", new ExecutionConfig()); SlotSharingGroup sharingGroup = new SlotSharingGroup(); JobVertex producer = new JobVertex("Speed Test Producer"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 45ee839bc1b97..09f9cacdc57a8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -22,6 +22,7 @@ import akka.actor.Kill; import akka.actor.PoisonPill; import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -167,7 +168,7 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { sender.setSlotSharingGroup(slotSharingGroup); receiver.setSlotSharingGroup(slotSharingGroup); - final JobGraph graph = new JobGraph("Blocking test job", sender, receiver); + final JobGraph graph = new JobGraph("Blocking test job", new ExecutionConfig(), sender, receiver); final ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration); diff --git a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java index 2586a27db3703..48888a4d161e2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java @@ -24,6 +24,7 @@ import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -178,7 +179,7 @@ public void testStop() throws Exception { sender.setParallelism(2); sender.setInvokableClass(StoppableInvokable.class); - final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", sender); + final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", new ExecutionConfig(), sender); final JobID jid = jobGraph.getJobID(); cluster.submitJobDetached(jobGraph); @@ -206,7 +207,7 @@ public void testStopYarn() throws Exception { sender.setParallelism(2); sender.setInvokableClass(StoppableInvokable.class); - final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", sender); + final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", new ExecutionConfig(), sender); final JobID jid = jobGraph.getJobID(); cluster.submitJobDetached(jobGraph); diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala index fddf63919a1f0..2265b3b58ab44 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala @@ -21,6 +21,7 @@ package org.apache.flink.api.scala.runtime.jobmanager import akka.actor.{ActorSystem, PoisonPill} import akka.testkit.{ImplicitSender, TestKit} import org.apache.flink.configuration.{ConfigConstants, Configuration} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, NoOpInvokable} @@ -93,12 +94,12 @@ class JobManagerFailsITCase(_system: ActorSystem) val sender = new JobVertex("BlockingSender") sender.setParallelism(num_slots) sender.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph("Blocking Testjob", sender) + val jobGraph = new JobGraph("Blocking Testjob", new ExecutionConfig(), sender) val noOp = new JobVertex("NoOpInvokable") noOp.setParallelism(num_slots) noOp.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph2 = new JobGraph("NoOp Testjob", noOp) + val jobGraph2 = new JobGraph("NoOp Testjob", new ExecutionConfig(), noOp) val cluster = startDeathwatchCluster(num_slots / 2, 2) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala index 09af4307bcfb2..9aa1e944682a4 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala @@ -23,6 +23,7 @@ import java.util.UUID import akka.actor.ActorSystem import akka.actor.Status.Success import akka.testkit.{ImplicitSender, TestKit} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils} import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex} @@ -64,7 +65,7 @@ class JobManagerLeaderSessionIDITSuite(_system: ActorSystem) val sender = new JobVertex("BlockingSender"); sender.setParallelism(numSlots) sender.setInvokableClass(classOf[BlockingUntilSignalNoOpInvokable]) - val jobGraph = new JobGraph("TestJob", sender) + val jobGraph = new JobGraph("TestJob", new ExecutionConfig(), sender) val oldSessionID = UUID.randomUUID() diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala index 869af82ed36df..88d760d17ac9a 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala @@ -18,9 +18,9 @@ package org.apache.flink.api.scala.runtime.taskmanager -import akka.actor.Status.{Failure, Success} import akka.actor.{ActorSystem, Kill, PoisonPill} import akka.testkit.{ImplicitSender, TestKit} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.configuration.ConfigConstants import org.apache.flink.configuration.Configuration @@ -33,7 +33,6 @@ import org.apache.flink.runtime.messages.TaskManagerMessages.{RegisteredAtJobMan import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils} -import org.apache.flink.runtime.util.SerializedThrowable import org.apache.flink.test.util.ForkableFlinkMiniCluster import org.junit.runner.RunWith @@ -100,7 +99,7 @@ class TaskManagerFailsITCase(_system: ActorSystem) receiver.setParallelism(num_tasks) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val jobID = jobGraph.getJobID val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2) @@ -152,7 +151,7 @@ class TaskManagerFailsITCase(_system: ActorSystem) receiver.setParallelism(num_tasks) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) val jobID = jobGraph.getJobID val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2) @@ -191,12 +190,12 @@ class TaskManagerFailsITCase(_system: ActorSystem) val sender = new JobVertex("BlockingSender") sender.setParallelism(num_slots) sender.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph("Blocking Testjob", sender) + val jobGraph = new JobGraph("Blocking Testjob", new ExecutionConfig(), sender) val noOp = new JobVertex("NoOpInvokable") noOp.setParallelism(num_slots) noOp.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph2 = new JobGraph("NoOp Testjob", noOp) + val jobGraph2 = new JobGraph("NoOp Testjob", new ExecutionConfig(), noOp) val cluster = createDeathwatchCluster(num_slots/2, 2) From 6406e03dc364e97ce8a8dc27f9d2c2e53f8836ad Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 3 Feb 2016 13:58:12 +0100 Subject: [PATCH 2/3] FLINK-3327: Attaches the ExecutionConfig to the JobGraph and propagates it to the Task itself. FLINK-3327: Attaches the ExecutionConfig to the JobGraph and propagates it to the Task itself. --- .../runtime/executiongraph/ExecutionStateProgressTest.java | 3 ++- .../org/apache/flink/runtime/jobmanager/JobManagerITCase.scala | 1 + .../apache/flink/runtime/jobmanager/SlotSharingITCase.scala | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java index d839ffb4a08c7..3317056320c54 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java @@ -85,4 +85,5 @@ public void testAccumulatedStateFinished() { fail(e.getMessage()); } } -} \ No newline at end of file +} + diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala index bf8df5ae96f60..ef036dd43000f 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala @@ -33,6 +33,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.{NoResourceAvailableExcepti import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils} +import org.apache.flink.runtime.testutils.JobManagerActorTestUtils import org.junit.runner.RunWith import org.mockito.Mockito._ import org.scalatest.junit.JUnitRunner diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala index 4dc80c383f796..4d320eabdadcb 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala @@ -19,6 +19,7 @@ package org.apache.flink.runtime.jobmanager import akka.actor.ActorSystem +import akka.actor.Status.Success import akka.testkit.{ImplicitSender, TestKit} import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.runtime.akka.ListeningBehaviour From 45d64c52345366830c135d0667e4b43786dc1156 Mon Sep 17 00:00:00 2001 From: kl0u Date: Wed, 17 Feb 2016 19:14:23 +0100 Subject: [PATCH 3/3] FLINK-2523: Makes the task cancellation interval configurable. --- docs/apis/common/index.md | 2 ++ .../flink/api/common/ExecutionConfig.java | 34 +++++++++++++++++-- .../flink/configuration/ConfigConstants.java | 5 +++ .../flink/runtime/taskmanager/Task.java | 13 ++++--- .../test/javaApiOperators/MapITCase.java | 2 ++ 5 files changed, 49 insertions(+), 7 deletions(-) diff --git a/docs/apis/common/index.md b/docs/apis/common/index.md index 6e1b27d3e8823..f7e63e6a40c34 100644 --- a/docs/apis/common/index.md +++ b/docs/apis/common/index.md @@ -999,6 +999,8 @@ Note that types registered with `registerKryoType()` are not available to Flink' - `disableAutoTypeRegistration()` Automatic type registration is enabled by default. The automatic type registration is registering all types (including sub-types) used by usercode with Kryo and the POJO serializer. +- `setTaskCancellationInterval(long interval)` Sets the the interval (in milliseconds) to wait between consecutive attempts to cancel a running task. When a task is canceled, a new thread is created, which calls periodically `interrupt()` on the task thread, if the task thread does not terminate within a certain time. This parameter refers to the time between consecutive calls to `interrupt()` and is set by default to **30000** milliseconds, or **30 seconds**. + The `RuntimeContext` which is accessible in `Rich*` functions through the `getRuntimeContext()` method also allows to access the `ExecutionConfig` in all user defined functions. {% top %} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index 2099b3d95b02b..cccc2db37593a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -22,6 +22,8 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.ConfigConstants; + import java.io.Serializable; import java.util.LinkedHashMap; @@ -108,7 +110,9 @@ public class ExecutionConfig implements Serializable { private long executionRetryDelay = DEFAULT_RESTART_DELAY; private RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration; - + + private long taskCancellationIntervalMillis = ConfigConstants.DEFAULT_TASK_CANCELLATION_INTERVAL_MILLIS; + // Serializers and types registered with Kryo and the PojoSerializer // we store them in linked maps/sets to ensure they are registered in order in all kryo instances. @@ -215,6 +219,28 @@ public ExecutionConfig setParallelism(int parallelism) { return this; } + /** + * Gets the interval (in milliseconds) between consecutive attempts to cancel a running task. + */ + public long getTaskCancellationInterval() { + return this.taskCancellationIntervalMillis; + } + + /** + * Sets the configuration parameter specifying the interval (in milliseconds) + * between consecutive attempts to cancel a running task. + * @param interval the interval (in milliseconds). + */ + public ExecutionConfig setTaskCancellationInterval(long interval) { + if(interval < 0) { + throw new IllegalArgumentException( + "The task cancellation interval cannot be negative." + ); + } + this.taskCancellationIntervalMillis = interval; + return this; + } + /** * Sets the restart strategy to be used for recovery. * @@ -661,7 +687,8 @@ public boolean equals(Object obj) { registeredTypesWithKryoSerializerClasses.equals(other.registeredTypesWithKryoSerializerClasses) && defaultKryoSerializerClasses.equals(other.defaultKryoSerializerClasses) && registeredKryoTypes.equals(other.registeredKryoTypes) && - registeredPojoTypes.equals(other.registeredPojoTypes); + registeredPojoTypes.equals(other.registeredPojoTypes) && + taskCancellationIntervalMillis == other.taskCancellationIntervalMillis; } else { return false; @@ -687,7 +714,8 @@ public int hashCode() { registeredTypesWithKryoSerializerClasses, defaultKryoSerializerClasses, registeredKryoTypes, - registeredPojoTypes); + registeredPojoTypes, + taskCancellationIntervalMillis); } public boolean canEqual(Object obj) { diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index f4e13f6b6e43e..38b684cdf4398 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -660,6 +660,11 @@ public final class ConfigConstants { */ public static final boolean DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE = false; + /** + * The default interval (in milliseconds) to wait between consecutive task cancellation attempts (= 30000 msec). + * */ + public static final long DEFAULT_TASK_CANCELLATION_INTERVAL_MILLIS = 30000; + // ------------------------ Runtime Algorithms ------------------------ /** 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 58e2153cea283..06bbf17ec184c 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 @@ -837,7 +837,9 @@ else if (current == ExecutionState.RUNNING) { // because the canceling may block on user code, we cancel from a separate thread // we do not reuse the async call handler, because that one may be blocked, in which // case the canceling could not continue - Runnable canceler = new TaskCanceler(LOG, invokable, executingThread, taskNameWithSubtask); + long taskCancellationInterval = this.executionConfig.getTaskCancellationInterval(); + Runnable canceler = new TaskCanceler(LOG, invokable, executingThread, taskNameWithSubtask, + taskCancellationInterval); Thread cancelThread = new Thread(executingThread.getThreadGroup(), canceler, "Canceler for " + taskNameWithSubtask); cancelThread.setDaemon(true); @@ -1081,12 +1083,15 @@ private static class TaskCanceler implements Runnable { private final AbstractInvokable invokable; private final Thread executer; private final String taskName; + private final long taskCancellationIntervalMillis; - public TaskCanceler(Logger logger, AbstractInvokable invokable, Thread executer, String taskName) { + public TaskCanceler(Logger logger, AbstractInvokable invokable, + Thread executer, String taskName, long cancelationInterval) { this.logger = logger; this.invokable = invokable; this.executer = executer; this.taskName = taskName; + this.taskCancellationIntervalMillis = cancelationInterval; } @Override @@ -1104,7 +1109,7 @@ public void run() { // interrupt the running thread initially executer.interrupt(); try { - executer.join(30000); + executer.join(taskCancellationIntervalMillis); } catch (InterruptedException e) { // we can ignore this @@ -1127,7 +1132,7 @@ public void run() { executer.interrupt(); try { - executer.join(30000); + executer.join(taskCancellationIntervalMillis); } catch (InterruptedException e) { // we can ignore this diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java index c1bd5e2fdd8ef..fb3e5890ae7cc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java @@ -76,6 +76,7 @@ public void testRuntimeContextAndExecutionConfigParams() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.getConfig().setNumberOfExecutionRetries(1000); + env.getConfig().setTaskCancellationInterval(50000); DataSet ds = CollectionDataSets.getStringDataSet(env); DataSet identityMapDs = ds. @@ -83,6 +84,7 @@ public void testRuntimeContextAndExecutionConfigParams() throws Exception { @Override public String map(String value) throws Exception { Assert.assertTrue(1000 == getRuntimeContext().getExecutionConfig().getNumberOfExecutionRetries()); + Assert.assertTrue(50000 == getRuntimeContext().getExecutionConfig().getTaskCancellationInterval()); return value; } });