From 17097fdf51f41445bad6da2186868185a6bf947b Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 18 Mar 2015 10:44:42 +0100 Subject: [PATCH 1/4] [FLINK-1679] deprecate API methods to set the parallelism --- .../flink/api/common/ExecutionConfig.java | 39 +++++++++++++- .../flink/api/common/operators/Operator.java | 41 ++++++++++++--- .../flink/api/java/CollectionEnvironment.java | 11 +++- .../flink/api/java/ExecutionEnvironment.java | 43 +++++++++++++-- .../api/scala/ExecutionEnvironment.scala | 26 ++++++++-- .../spargel/java/record/SpargelIteration.java | 14 +++-- .../StreamExecutionEnvironment.java | 52 +++++++++++++++---- .../scala/StreamExecutionEnvironment.scala | 26 ++++++++-- 8 files changed, 217 insertions(+), 35 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 aa025c1186bed..a2df4384d956a 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 @@ -137,8 +137,25 @@ public boolean isClosureCleanerEnabled() { * * @return The parallelism used by operations, unless they override that value. This method * returns {@code -1}, if the environment's default parallelism should be used. + * @deprecated Please use {@link #getParallelism} */ + @Deprecated public int getDegreeOfParallelism() { + return getParallelism(); + } + + /** + * Gets the parallelism with which operation are executed by default. Operations can + * individually override this value to use a specific parallelism. + * + * Other operations may need to run with a different parallelism - for example calling + * a reduce operation over the entire data set will involve an operation that runs + * with a parallelism of one (the final reduce to the single result value). + * + * @return The parallelism used by operations, unless they override that value. This method + * returns {@code -1}, if the environment's default parallelism should be used. + */ + public int getParallelism() { return parallelism; } @@ -150,14 +167,32 @@ public int getDegreeOfParallelism() { * This method overrides the default parallelism for this environment. * The local execution environment uses by default a value equal to the number of hardware * contexts (CPU cores / threads). When executing the program via the command line client - * from a JAR file, the default degree of parallelism is the one configured for that setup. + * from a JAR file, the default parallelism is the one configured for that setup. * * @param parallelism The parallelism to use + * @deprecated Please use {@link #setParallelism} */ + @Deprecated public ExecutionConfig setDegreeOfParallelism(int parallelism) { + return setParallelism(parallelism); + } + + /** + * Sets the parallelism for operations executed through this environment. + * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with + * x parallel instances. + *

+ * This method overrides the default parallelism for this environment. + * The local execution environment uses by default a value equal to the number of hardware + * contexts (CPU cores / threads). When executing the program via the command line client + * from a JAR file, the default parallelism is the one configured for that setup. + * + * @param parallelism The parallelism to use + */ + public ExecutionConfig setParallelism(int parallelism) { if (parallelism < 1 && parallelism != -1) { throw new IllegalArgumentException( - "Degree of parallelism must be at least one, or -1 (use system default)."); + "Parallelism must be at least one, or -1 (use system default)."); } this.parallelism = parallelism; return this; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java index 85b352a901039..840c253eab21a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java @@ -40,7 +40,7 @@ public abstract class Operator implements Visitable> { protected String name; // the name of the contract instance. optional. - private int degreeOfParallelism = -1; // the number of parallel instances to use. -1, if unknown + private int parallelism = -1; // the number of parallel instances to use. -1, if unknown /** * The return type of the user function. @@ -160,25 +160,50 @@ public void setParameter(String key, boolean value) { } /** - * Gets the degree of parallelism for this contract instance. The degree of parallelism denotes + * Gets the parallelism for this contract instance. The parallelism denotes * how many parallel instances of the user function will be spawned during the execution. If this * value is -1, then the system will decide the number of parallel instances by itself. * - * @return The degree of parallelism. + * @return The parallelism. + * @deprecated Please use {@link #getParallelism} */ + @Deprecated public int getDegreeOfParallelism() { - return this.degreeOfParallelism; + return getParallelism(); } /** - * Sets the degree of parallelism for this contract instance. The degree of parallelism denotes + * Gets the parallelism for this contract instance. The parallelism denotes + * how many parallel instances of the user function will be spawned during the execution. If this + * value is -1, then the system will decide the number of parallel instances by itself. + * + * @return The parallelism. + */ + public int getParallelism() { + return this.parallelism; + } + + /** + * Sets the parallelism for this contract instance. The parallelism denotes * how many parallel instances of the user function will be spawned during the execution. Set this * value to -1 to let the system decide on its own. * - * @param degree The number of parallel instances to spawn. -1, if unspecified. + * @param parallelism The number of parallel instances to spawn. -1, if unspecified. + * @deprecated Please use {@link #setParallelism} + */ + @Deprecated + public void setDegreeOfParallelism(int parallelism) { + setParallelism(parallelism); + } + /** + * Sets the parallelism for this contract instance. The parallelism denotes + * how many parallel instances of the user function will be spawned during the execution. Set this + * value to -1 to let the system decide on its own. + * + * @param parallelism The number of parallel instances to spawn. -1, if unspecified. */ - public void setDegreeOfParallelism(int degree) { - this.degreeOfParallelism = degree; + public void setParallelism(int parallelism) { + this.parallelism = parallelism; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java index c61d624079d0d..b48debc381cd5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java @@ -32,9 +32,18 @@ public JobExecutionResult execute(String jobName) throws Exception { CollectionExecutor exec = new CollectionExecutor(getConfig()); return exec.execute(p); } - + + /** + * @deprecated Please use {@link #getParallelism} + */ @Override + @Deprecated public int getDegreeOfParallelism() { + return getParallelism(); + } + + @Override + public int getParallelism() { return 1; // always serial } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index 929bf6ea0ae9b..aac3147f8f6a5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -142,9 +142,26 @@ public ExecutionConfig getConfig() { * * @return The degree of parallelism used by operations, unless they override that value. This method * returns {@code -1}, if the environments default parallelism should be used. + * @deprecated Please use {@link #getParallelism} */ + @Deprecated public int getDegreeOfParallelism() { - return config.getDegreeOfParallelism(); + return getParallelism(); + } + + /** + * Gets the parallelism with which operation are executed by default. Operations can + * individually override this value to use a specific parallelism via + * {@link Operator#setParallelism(int)}. Other operations may need to run with a different + * parallelism - for example calling + * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire + * set will insert eventually an operation that runs non-parallel (parallelism of one). + * + * @return The parallelism used by operations, unless they override that value. This method + * returns {@code -1}, if the environments default parallelism should be used. + */ + public int getParallelism() { + return config.getParallelism(); } /** @@ -157,10 +174,28 @@ public int getDegreeOfParallelism() { * contexts (CPU cores / threads). When executing the program via the command line client * from a JAR file, the default degree of parallelism is the one configured for that setup. * - * @param degreeOfParallelism The degree of parallelism + * @param parallelism The parallelism + * @deprecated Please use {@link #setParallelism} + */ + @Deprecated + public void setDegreeOfParallelism(int parallelism) { + setParallelism(parallelism); + } + + /** + * Sets the parallelism for operations executed through this environment. + * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with + * x parallel instances. + *

+ * This method overrides the default parallelism for this environment. + * The {@link LocalEnvironment} uses by default a value equal to the number of hardware + * contexts (CPU cores / threads). When executing the program via the command line client + * from a JAR file, the default parallelism is the one configured for that setup. + * + * @param parallelism The parallelism */ - public void setDegreeOfParallelism(int degreeOfParallelism) { - config.setDegreeOfParallelism(degreeOfParallelism); + public void setParallelism(int parallelism) { + config.setParallelism(parallelism); } /** diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala index f49aef1dec408..cccea78f13102 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala @@ -82,17 +82,35 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with * x parallel instances. This value can be overridden by specific operations using * [[DataSet.setParallelism]]. + * @deprecated Please use [[setParallelism]] */ - def setDegreeOfParallelism(degreeOfParallelism: Int): Unit = { - javaEnv.setDegreeOfParallelism(degreeOfParallelism) + @deprecated + def setDegreeOfParallelism(parallelism: Int): Unit = { + setParallelism(parallelism) } /** * Returns the default degree of parallelism for this execution environment. Note that this * value can be overridden by individual operations using [[DataSet.setParallelism] */ - def getDegreeOfParallelism = javaEnv.getDegreeOfParallelism - + def setParallelism(parallelism: Int): Unit = { + javaEnv.setParallelism(parallelism) + } + + /** + * Returns the default parallelism for this execution environment. Note that this + * value can be overridden by individual operations using [[DataSet.setParallelism]] + * @deprecated Please use [[getParallelism]] + */ + @deprecated + def getDegreeOfParallelism = javaEnv.getParallelism + + /** + * Returns the default parallelism for this execution environment. Note that this + * value can be overridden by individual operations using [[DataSet.setParallelism]] + */ + def getParallelism = javaEnv.getParallelism + /** * Sets the number of times that failed tasks are re-executed. A value of zero * effectively disables fault tolerance. A value of "-1" indicates that the system diff --git a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java index f647e5d95588f..8f1839d2a1dc8 100644 --- a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java +++ b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java @@ -127,9 +127,17 @@ public void setEdgesInput(Operator c) { public Operator getOutput() { return this.iteration; } - - public void setDegreeOfParallelism(int dop) { - this.iteration.setDegreeOfParallelism(dop); + + /** + * @deprecated Please use {@link #setParallelism} + */ + @Deprecated + public void setDegreeOfParallelism(int parallelism) { + setParallelism(parallelism); + } + + public void setParallelism(int parallelism) { + this.iteration.setParallelism(parallelism); } public void setNumberOfIterations(int iterations) { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 2096745be0759..ae4e19bf6d9b2 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -97,16 +97,30 @@ public ExecutionConfig getConfig() { * default. Operations can individually override this value to use a * specific degree of parallelism. * - * @return The degree of parallelism used by operations, unless they + * @return The parallelism used by operations, unless they * override that value. + * @deprecated Please use {@link #getParallelism} */ + @Deprecated public int getDegreeOfParallelism() { - return config.getDegreeOfParallelism(); + return getParallelism(); } /** - * Sets the degree of parallelism (DOP) for operations executed through this - * environment. Setting a DOP of x here will cause all operators (such as + * Gets the parallelism with which operation are executed by + * default. Operations can individually override this value to use a + * specific parallelism. + * + * @return The parallelism used by operations, unless they + * override that value. + */ + public int getParallelism() { + return config.getParallelism(); + } + + /** + * Sets the parallelism for operations executed through this + * environment. Setting a parallelism of x here will cause all operators (such as * map, batchReduce) to run with x parallel instances. This method overrides * the default parallelism for this environment. The * {@link LocalStreamEnvironment} uses by default a value equal to the @@ -114,14 +128,32 @@ public int getDegreeOfParallelism() { * program via the command line client from a JAR file, the default degree * of parallelism is the one configured for that setup. * - * @param degreeOfParallelism - * The degree of parallelism + * @param parallelism The parallelism + * @deprecated Please use {@link #setParallelism} + */ + @Deprecated + public StreamExecutionEnvironment setDegreeOfParallelism(int parallelism) { + return setParallelism(parallelism); + } + + /** + * Sets the parallelism for operations executed through this + * environment. Setting a parallelism of x here will cause all operators (such as + * map, batchReduce) to run with x parallel instances. This method overrides + * the default parallelism for this environment. The + * {@link LocalStreamEnvironment} uses by default a value equal to the + * number of hardware contexts (CPU cores / threads). When executing the + * program via the command line client from a JAR file, the default degree + * of parallelism is the one configured for that setup. + * + * @param parallelism + * The parallelism */ - public StreamExecutionEnvironment setDegreeOfParallelism(int degreeOfParallelism) { - if (degreeOfParallelism < 1) { - throw new IllegalArgumentException("Degree of parallelism must be at least one."); + public StreamExecutionEnvironment setParallelism(int parallelism) { + if (parallelism < 1) { + throw new IllegalArgumentException("parallelism must be at least one."); } - config.setDegreeOfParallelism(degreeOfParallelism); + config.setParallelism(parallelism); return this; } diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala index 4672fca1f338d..7596e6d297eec 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala @@ -37,16 +37,36 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with * x parallel instances. This value can be overridden by specific operations using * [[DataStream.setParallelism]]. + * @deprecated Please use [[setParallelism]] */ + @deprecated def setDegreeOfParallelism(degreeOfParallelism: Int): Unit = { - javaEnv.setDegreeOfParallelism(degreeOfParallelism) + javaEnv.setParallelism(degreeOfParallelism) + } + + /** + * Sets the parallelism for operations executed through this environment. + * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run + * with x parallel instances. This value can be overridden by specific operations using + * [[DataStream.setParallelism]]. + */ + def setParallelism(parallelism: Int): Unit = { + javaEnv.setParallelism(parallelism) } /** - * Returns the default degree of parallelism for this execution environment. Note that this + * Returns the default parallelism for this execution environment. Note that this + * value can be overridden by individual operations using [[DataStream.setParallelism]] + * @deprecated Please use [[getParallelism]] + */ + @deprecated + def getDegreeOfParallelism = javaEnv.getParallelism + + /** + * Returns the default parallelism for this execution environment. Note that this * value can be overridden by individual operations using [[DataStream.setParallelism]] */ - def getDegreeOfParallelism = javaEnv.getDegreeOfParallelism + def getParallelism = javaEnv.getParallelism /** * Sets the maximum time frequency (milliseconds) for the flushing of the From f6ba8c07cc9a153b1ac1e213f9749155c42ae3c3 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 18 Mar 2015 10:44:43 +0100 Subject: [PATCH 2/4] [FLINK-1679] use a consistent name for parallelism * rename occurrences of degree of parallelism to parallelism * [Dd]egree[ -]of[ -]parallelism -> [pP]arallelism * (DOP|dop) -> [pP]arallelism * paraDegree -> parallelism * degree-of-parallelism -> parallelism * DEGREE_OF_PARALLELISM -> PARALLELISM --- docs/cluster_setup.md | 2 +- docs/config.md | 4 +- docs/gelly_guide.md | 6 +- docs/internal_job_scheduling.md | 2 +- docs/programming_guide.md | 28 +++--- docs/setup_quickstart.md | 2 +- docs/streaming_guide.md | 6 +- docs/yarn_setup.md | 2 +- .../org/apache/flink/client/CliFrontend.java | 16 ++-- .../flink/client/FlinkYarnSessionCli.java | 2 +- .../apache/flink/client/program/Client.java | 2 +- .../client/program/ContextEnvironment.java | 8 +- .../flink/api/common/ExecutionConfig.java | 2 +- .../org/apache/flink/api/common/Plan.java | 22 ++--- .../api/common/functions/RuntimeContext.java | 8 +- .../flink/api/common/io/FinalizeOnMaster.java | 2 +- .../api/common/io/InitializeOnMaster.java | 2 +- .../flink/api/common/io/NonParallelInput.java | 2 +- .../api/common/io/ReplicatingInputFormat.java | 4 +- .../flink/configuration/ConfigConstants.java | 8 +- .../common/io/SequentialFormatTestBase.java | 34 +++---- .../api/common/io/SerializedFormatTest.java | 4 +- flink-dist/src/main/resources/flink-conf.yaml | 2 +- .../org/apache/flink/api/java/DataSet.java | 2 +- .../flink/api/java/ExecutionEnvironment.java | 64 ++++++------- .../flink/api/java/LocalEnvironment.java | 6 +- .../flink/api/java/RemoteEnvironment.java | 10 +- .../flink/api/java/SortPartitionOperator.java | 4 +- .../api/java/io/PrintingOutputFormat.java | 2 +- .../api/java/operators/AggregateOperator.java | 6 +- .../api/java/operators/CoGroupOperator.java | 20 ++-- .../api/java/operators/CrossOperator.java | 2 +- .../flink/api/java/operators/DataSink.java | 32 +++---- .../flink/api/java/operators/DataSource.java | 4 +- .../api/java/operators/DeltaIteration.java | 8 +- .../api/java/operators/DistinctOperator.java | 6 +- .../api/java/operators/FilterOperator.java | 10 +- .../api/java/operators/FlatMapOperator.java | 10 +- .../java/operators/GroupCombineOperator.java | 14 +-- .../java/operators/GroupReduceOperator.java | 14 +-- .../api/java/operators/JoinOperator.java | 30 +++--- .../flink/api/java/operators/MapOperator.java | 10 +- .../java/operators/MapPartitionOperator.java | 10 +- .../flink/api/java/operators/Operator.java | 20 ++-- .../java/operators/OperatorTranslation.java | 2 +- .../api/java/operators/PartitionOperator.java | 12 +-- .../api/java/operators/ProjectOperator.java | 4 +- .../api/java/operators/ReduceOperator.java | 16 ++-- .../api/java/io/TypeSerializerFormatTest.java | 4 +- .../translation/AggregateTranslationTest.java | 6 +- .../DeltaIterationTranslationTest.java | 12 +-- .../translation/DistinctTranslationTest.java | 42 ++++---- .../translation/ReduceTranslationTests.java | 28 +++--- .../org/apache/flink/optimizer/Optimizer.java | 30 +++--- .../flink/optimizer/dag/BinaryUnionNode.java | 18 ++-- .../optimizer/dag/BulkIterationNode.java | 6 +- .../flink/optimizer/dag/DataSinkNode.java | 4 +- .../flink/optimizer/dag/DataSourceNode.java | 6 +- .../flink/optimizer/dag/GroupCombineNode.java | 2 +- .../flink/optimizer/dag/GroupReduceNode.java | 2 +- .../flink/optimizer/dag/OptimizerNode.java | 6 +- .../flink/optimizer/dag/ReduceNode.java | 2 +- .../flink/optimizer/dag/SingleInputNode.java | 21 ++-- .../flink/optimizer/dag/SinkJoiner.java | 2 +- .../flink/optimizer/dag/TwoInputNode.java | 22 ++--- .../optimizer/dag/WorksetIterationNode.java | 8 +- .../RequestedGlobalProperties.java | 2 +- ...AllGroupWithPartialPreGroupProperties.java | 4 +- .../operators/AllReduceProperties.java | 4 +- .../operators/GroupCombineProperties.java | 2 +- .../GroupReduceWithCombineProperties.java | 4 +- .../operators/PartialGroupProperties.java | 4 +- .../optimizer/operators/ReduceProperties.java | 4 +- .../apache/flink/optimizer/plan/Channel.java | 2 +- .../optimizer/plan/SinkJoinerPlanNode.java | 2 +- .../plandump/PlanJSONDumpGenerator.java | 2 +- .../plantranslate/JobGraphGenerator.java | 24 ++--- .../traversals/GraphCreatingVisitor.java | 24 ++--- .../CachedMatchStrategyCompilerTest.java | 4 +- .../flink/optimizer/CompilerTestBase.java | 4 +- .../optimizer/DistinctCompilationTest.java | 12 +-- .../optimizer/IterationsCompilerTest.java | 10 +- ...geTest.java => ParallelismChangeTest.java} | 96 +++++++++---------- .../flink/optimizer/PipelineBreakerTest.java | 14 +-- .../optimizer/PropertyDataSourceTest.java | 48 +++++----- .../optimizer/ReplicatingDataSourceTest.java | 32 +++---- .../CustomPartitioningTest.java | 12 +-- .../DistinctAndGroupingOptimizerTest.java | 4 +- .../java/GroupReduceCompilationTest.java | 26 ++--- .../optimizer/java/IterationCompilerTest.java | 8 +- .../optimizer/java/ReduceCompilationTest.java | 18 ++-- .../WorksetIterationsJavaApiCompilerTest.java | 4 +- .../executiongraph/ExecutionGraph.java | 2 +- .../runtime/jobgraph/AbstractJobVertex.java | 10 +- .../operators/shipping/ShipStrategyType.java | 2 +- .../executiongraph/LocalInputSplitsTest.java | 2 +- .../operators/ScalaAggregateOperator.java | 8 +- .../org/apache/flink/api/scala/DataSet.scala | 14 +-- .../api/scala/ExecutionEnvironment.scala | 34 +++---- .../graph/spargel/VertexCentricIteration.java | 8 +- .../DegreesWithExceptionITCase.java | 10 +- .../mapred/HadoopMapredITCase.java | 2 +- .../mapreduce/HadoopInputOutputITCase.java | 2 +- .../ml/feature/PolynomialBaseITCase.scala | 6 +- .../flink/ml/recommendation/ALSITCase.scala | 2 +- .../MultipleLinearRegressionITCase.scala | 4 +- .../spargel/java/VertexCentricIteration.java | 8 +- .../spargel/java/SpargelCompilerTest.java | 4 +- .../spargel/java/SpargelTranslationTest.java | 12 +-- .../kafka/KafkaConsumerExample.java | 2 +- .../kafka/KafkaProducerExample.java | 2 +- .../kafka/KafkaSimpleConsumerExample.java | 2 +- .../connectors/kafka/KafkaITCase.java | 2 +- .../api/datastream/ConnectedDataStream.java | 2 +- .../streaming/api/datastream/DataStream.java | 10 +- .../api/datastream/DataStreamSource.java | 6 +- .../api/datastream/DiscretizedStream.java | 2 +- .../SingleOutputStreamOperator.java | 16 ++-- .../api/datastream/WindowedDataStream.java | 4 +- .../environment/LocalStreamEnvironment.java | 4 +- .../environment/RemoteStreamEnvironment.java | 4 +- .../environment/StreamContextEnvironment.java | 12 +-- .../StreamExecutionEnvironment.java | 54 +++++------ .../environment/StreamPlanEnvironment.java | 12 +-- .../api/function/sink/PrintSinkFunction.java | 2 +- .../api/function/source/SourceFunction.java | 2 +- .../flink/streaming/util/ClusterUtil.java | 6 +- .../util/StreamingProgramTestBase.java | 18 ++-- .../streaming/util/TestStreamEnvironment.java | 10 +- .../streaming/api/scala/DataStream.scala | 8 +- .../scala/StreamExecutionEnvironment.scala | 26 ++--- .../org/apache/flink/tachyon/HDFSTest.java | 2 +- .../tachyon/TachyonFileSystemWrapperTest.java | 4 +- .../test/compiler/util/CompilerTestBase.java | 4 +- .../flink/test/util/JavaProgramTestBase.java | 22 ++--- .../flink/test/util/RecordAPITestBase.java | 4 +- .../flink/test/util/TestEnvironment.java | 4 +- .../test/accumulators/AccumulatorITCase.java | 2 +- .../BroadcastBranchingITCase.java | 2 +- .../BroadcastVarInitializationITCase.java | 2 +- .../BroadcastVarsNepheleITCase.java | 6 +- .../KMeansIterativeNepheleITCase.java | 12 +-- .../test/cancelling/MapCancelingITCase.java | 12 +-- .../cancelling/MatchJoinCancelingITCase.java | 18 ++-- .../clients/examples/LocalExecutorITCase.java | 6 +- .../exampleScalaPrograms/WordCountITCase.java | 2 +- .../failingPrograms/TaskFailureITCase.java | 6 +- .../CoGroupConnectedComponentsITCase.java | 4 +- .../iterative/ConnectedComponentsITCase.java | 4 +- ...tedComponentsWithDeferredUpdateITCase.java | 4 +- ...dComponentsWithSolutionSetFirstITCase.java | 4 +- .../iterative/DanglingPageRankITCase.java | 2 +- ...rationNotDependingOnSolutionSetITCase.java | 2 +- .../test/iterative/DeltaPageRankITCase.java | 2 +- .../DependencyConnectedComponentsITCase.java | 6 +- ...erationTerminationWithTerminationTail.java | 4 +- .../IterationTerminationWithTwoTails.java | 4 +- .../IterationWithAllReducerITCase.java | 2 +- .../IterationWithChainingITCase.java | 6 +- .../test/iterative/IterativeKMeansITCase.java | 4 +- .../flink/test/iterative/KMeansITCase.java | 4 +- .../flink/test/iterative/LineRankITCase.java | 4 +- .../flink/test/iterative/PageRankITCase.java | 2 +- .../aggregators/AggregatorsITCase.java | 12 +-- ...ntsWithParametrizableAggregatorITCase.java | 4 +- ...tsWithParametrizableConvergenceITCase.java | 4 +- .../ConnectedComponentsNepheleITCase.java | 14 +-- .../DanglingPageRankNepheleITCase.java | 4 +- ...lingPageRankWithCombinerNepheleITCase.java | 4 +- .../IterationWithChainingNepheleITCase.java | 4 +- .../test/iterative/nephele/JobGraphUtils.java | 8 +- .../CustomCompensatableDanglingPageRank.java | 20 ++-- ...ensatableDanglingPageRankWithCombiner.java | 20 ++-- .../CompensatableDanglingPageRank.java | 20 ++-- .../test/javaApiOperators/DataSinkITCase.java | 22 ++--- .../javaApiOperators/GroupCombineITCase.java | 4 +- .../javaApiOperators/GroupReduceITCase.java | 32 +++---- .../test/javaApiOperators/JoinITCase.java | 14 +-- .../javaApiOperators/PartitionITCase.java | 12 +-- .../javaApiOperators/SortPartitionITCase.java | 20 ++-- .../test/misc/AutoParallelismITCase.java | 2 +- .../test/misc/CustomPartitioningITCase.java | 2 +- .../flink/test/misc/NullValuesITCase.java | 2 +- .../flink/test/operators/CoGroupITCase.java | 8 +- .../flink/test/operators/CrossITCase.java | 8 +- .../flink/test/operators/JoinITCase.java | 8 +- .../flink/test/operators/MapITCase.java | 6 +- .../flink/test/operators/ReduceITCase.java | 6 +- .../flink/test/operators/UnionITCase.java | 8 +- .../flink/test/operators/UnionSinkITCase.java | 4 +- .../recordJobTests/CollectionSourceTest.java | 6 +- .../ComputeEdgeDegreesITCase.java | 4 +- ...EnumTrianglesOnEdgesWithDegreesITCase.java | 4 +- .../EnumTrianglesRDFITCase.java | 4 +- .../recordJobTests/GlobalSortingITCase.java | 8 +- .../GlobalSortingMixedOrderITCase.java | 4 +- .../GroupOrderReduceITCase.java | 8 +- .../test/recordJobTests/PairwiseSPITCase.java | 4 +- .../recordJobTests/TPCHQuery10ITCase.java | 2 +- .../test/recordJobTests/TPCHQuery3ITCase.java | 6 +- .../TPCHQuery3WithUnionITCase.java | 4 +- .../test/recordJobTests/TPCHQuery4ITCase.java | 4 +- .../test/recordJobTests/TPCHQuery9ITCase.java | 4 +- .../TPCHQueryAsterixITCase.java | 4 +- .../test/recordJobTests/TeraSortITCase.java | 4 +- .../recordJobTests/WebLogAnalysisITCase.java | 4 +- .../test/recordJobTests/WordCountITCase.java | 4 +- .../WordCountUnionReduceITCase.java | 4 +- .../recordJobs/graph/DanglingPageRank.java | 8 +- .../test/recordJobs/graph/PairwiseSP.java | 8 +- .../test/recordJobs/graph/SimplePageRank.java | 8 +- .../recordJobs/kmeans/KMeansBroadcast.java | 4 +- .../test/recordJobs/kmeans/KMeansCross.java | 2 +- .../recordJobs/relational/MergeOnlyJoin.java | 4 +- .../recordJobs/relational/TPCHQuery1.java | 14 +-- .../recordJobs/relational/TPCHQuery10.java | 6 +- .../recordJobs/relational/TPCHQuery4.java | 20 ++-- .../recordJobs/relational/TPCHQuery9.java | 10 +- .../relational/TPCHQueryAsterix.java | 10 +- .../recordJobs/relational/WebLogAnalysis.java | 2 +- .../flink/test/recordJobs/sort/TeraSort.java | 4 +- .../ProcessFailureBatchRecoveryITCase.java | 2 +- .../test/recovery/SimpleRecoveryITCase.java | 8 +- .../TaskManagerFailureRecoveryITCase.java | 2 +- ...nsumePipelinedAndBlockingResultITCase.java | 2 +- .../test/util/testjar/KMeansForTest.java | 2 +- .../scala/operators/GroupReduceITCase.scala | 28 +++--- .../api/scala/operators/JoinITCase.scala | 12 +-- .../api/scala/operators/PartitionITCase.scala | 24 ++--- .../AggregateTranslationTest.scala | 6 +- .../translation/CustomPartitioningTest.scala | 10 +- .../DeltaIterationTranslationTest.scala | 12 +-- .../translation/ReduceTranslationTest.scala | 22 ++--- .../apache/flink/yarn/ApplicationMaster.scala | 2 +- 234 files changed, 1085 insertions(+), 1082 deletions(-) rename flink-optimizer/src/test/java/org/apache/flink/optimizer/{DOPChangeTest.java => ParallelismChangeTest.java} (85%) diff --git a/docs/cluster_setup.md b/docs/cluster_setup.md index d425be88408c7..2ee9f3c05be94 100644 --- a/docs/cluster_setup.md +++ b/docs/cluster_setup.md @@ -323,7 +323,7 @@ In particular, * the amount of available memory per TaskManager (`taskmanager.heap.mb`), * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`), - * the total number of CPUs in the cluster (`parallelization.degree.default`) and + * the total number of CPUs in the cluster (`parallelism.default`) and * the temporary directories (`taskmanager.tmp.dirs`) are very important configuration values. diff --git a/docs/config.md b/docs/config.md index b8cf06a6ad588..09a01d9d17a8f 100644 --- a/docs/config.md +++ b/docs/config.md @@ -80,8 +80,8 @@ This value is typically proportional to the number of physical CPU cores that the TaskManager's machine has (e.g., equal to the number of cores, or half the number of cores). [More about task slots](config.html#configuring-taskmanager-processing-slots). -- `parallelization.degree.default`: The default degree of parallelism to use for -programs that have no degree of parallelism specified. (DEFAULT: 1). For +- `parallelism.default`: The default parallelism to use for +programs that have no parallelism specified. (DEFAULT: 1). For setups that have no concurrent jobs running, setting this value to NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all available execution resources for the program's execution. diff --git a/docs/gelly_guide.md b/docs/gelly_guide.md index 8058e0ae8c973..32c076bb4c95f 100644 --- a/docs/gelly_guide.md +++ b/docs/gelly_guide.md @@ -344,7 +344,7 @@ Gelly wraps Flink's [Spargel API](spargel_guide.html) to provide methods for ver Like in Spargel, the user only needs to implement two functions: a `VertexUpdateFunction`, which defines how a vertex will update its value based on the received messages and a `MessagingFunction`, which allows a vertex to send out messages for the next superstep. These functions are given as parameters to Gelly's `createVertexCentricIteration`, which returns a `VertexCentricIteration`. -The user can configure this iteration (set the name, the degree of parallelism, aggregators, etc.) and then run the computation, using the `runVertexCentricIteration` method: +The user can configure this iteration (set the name, the parallelism, aggregators, etc.) and then run the computation, using the `runVertexCentricIteration` method: {% highlight java %} Graph graph = ... @@ -357,8 +357,8 @@ VertexCentricIteration iteration = // set the iteration name iteration.setName("Single Source Shortest Paths"); -// set the degree of parallelism -iteration.setDegreeOfParallelism(16); +// set the parallelism +iteration.setParallelism(16); // run the computation graph.runVertexCentricIteration(iteration); diff --git a/docs/internal_job_scheduling.md b/docs/internal_job_scheduling.md index 0d08f76bd8d5e..2c55c016a5882 100644 --- a/docs/internal_job_scheduling.md +++ b/docs/internal_job_scheduling.md @@ -57,7 +57,7 @@ and reacts to finished tasks or execution failures. The JobManager receives the {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ "JobGraph" %}, which is a representation of the data flow consisting of operators ({% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java "JobVertex" %}) and intermediate results ({% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java "IntermediateDataSet" %}). -Each operator has properies, like the degree of parallelism and the code that it executes. +Each operator has properies, like the parallelism and the code that it executes. In addition, the JobGraph has a set of attached libraries, that are neccessary to execute the code of the operators. The JobManager transforms the JobGraph into an {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ "ExecutionGraph" %}. diff --git a/docs/programming_guide.md b/docs/programming_guide.md index e1c79ee0eb2cd..1750a12c71ae3 100644 --- a/docs/programming_guide.md +++ b/docs/programming_guide.md @@ -218,10 +218,10 @@ obtain one using these static methods on class `ExecutionEnvironment`: getExecutionEnvironment() createLocalEnvironment() -createLocalEnvironment(int degreeOfParallelism) +createLocalEnvironment(int parallelism) createRemoteEnvironment(String host, int port, String... jarFiles) -createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles) +createRemoteEnvironment(String host, int port, int parallelism, String... jarFiles) {% endhighlight %} Typically, you only need to use `getExecutionEnvironment()`, since this @@ -318,10 +318,10 @@ obtain one using these static methods on class `ExecutionEnvironment`: {% highlight scala %} def getExecutionEnvironment -def createLocalEnvironment(degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors())) +def createLocalEnvironment(parallelism: Int = Runtime.getRuntime.availableProcessors())) def createRemoteEnvironment(host: String, port: String, jarFiles: String*) -def createRemoteEnvironment(host: String, port: String, degreeOfParallelism: Int, jarFiles: String*) +def createRemoteEnvironment(host: String, port: String, parallelism: Int, jarFiles: String*) {% endhighlight %} Typically, you only need to use `getExecutionEnvironment()`, since this @@ -2074,7 +2074,7 @@ val myLongs = env.fromCollection(longIt) **Note:** Currently, the collection data source requires that data types and iterators implement -`Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of +`Serializable`. Furthermore, collection data sources can not be executed in parallel ( parallelism = 1). [Back to top](#top) @@ -2704,15 +2704,15 @@ Parallel Execution This section describes how the parallel execution of programs can be configured in Flink. A Flink program consists of multiple tasks (operators, data sources, and sinks). A task is split into several parallel instances for execution and each parallel instance processes a subset of the task's -input data. The number of parallel instances of a task is called its *parallelism* or *degree of -parallelism (DOP)*. +input data. The number of parallel instances of a task is called its *parallelism*. -The degree of parallelism of a task can be specified in Flink on different levels. + +The parallelism of a task can be specified in Flink on different levels. ### Operator Level The parallelism of an individual operator, data source, or data sink can be defined by calling its -`setParallelism()` method. For example, the degree of parallelism of the `Sum` operator in the +`setParallelism()` method. For example, the parallelism of the `Sum` operator in the [WordCount](#example-program) example program can be set to `5` as follows : @@ -2749,13 +2749,13 @@ env.execute("Word Count Example") ### Execution Environment Level -Flink programs are executed in the context of an [execution environmentt](#program-skeleton). An +Flink programs are executed in the context of an [execution environment](#program-skeleton). An execution environment defines a default parallelism for all operators, data sources, and data sinks it executes. Execution environment parallelism can be overwritten by explicitly configuring the parallelism of an operator. The default parallelism of an execution environment can be specified by calling the -`setDegreeOfParallelism()` method. To execute all operators, data sources, and data sinks of the +`setParallelism()` method. To execute all operators, data sources, and data sinks of the [WordCount](#example-program) example program with a parallelism of `3`, set the default parallelism of the execution environment as follows: @@ -2763,7 +2763,7 @@ execution environment as follows:

{% highlight java %} final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); -env.setDegreeOfParallelism(3); +env.setParallelism(3); DataSet text = [...] DataSet> wordCounts = [...] @@ -2775,7 +2775,7 @@ env.execute("Word Count Example");
{% highlight scala %} val env = ExecutionEnvironment.getExecutionEnvironment -env.setDegreeOfParallelism(3) +env.setParallelism(3) val text = [...] val wordCounts = text @@ -2792,7 +2792,7 @@ env.execute("Word Count Example") ### System Level A system-wide default parallelism for all execution environments can be defined by setting the -`parallelization.degree.default` property in `./conf/flink-conf.yaml`. See the +`parallelism.default` property in `./conf/flink-conf.yaml`. See the [Configuration](config.html) documentation for details. [Back to top](#top) diff --git a/docs/setup_quickstart.md b/docs/setup_quickstart.md index e45792d0772e6..5a19d836c16f5 100644 --- a/docs/setup_quickstart.md +++ b/docs/setup_quickstart.md @@ -139,7 +139,7 @@ In particular, * the amount of available memory per TaskManager (`taskmanager.heap.mb`), * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`), - * the total number of CPUs in the cluster (`parallelization.degree.default`) and + * the total number of CPUs in the cluster (`parallelism.default`) and * the temporary directories (`taskmanager.tmp.dirs`) diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md index 6d62b75a946ed..85edc1616a2c2 100644 --- a/docs/streaming_guide.md +++ b/docs/streaming_guide.md @@ -281,7 +281,7 @@ The user is expected to connect to the outside world through the source and the The user can connect to data streams by the different implementations of `SourceFunction` using `StreamExecutionEnvironment.addSource(sourceFunction)`. In contrast with other operators, DataStreamSources have a default operator parallelism of 1. -To create parallel sources the users source function needs to implement `ParallelSourceFunction` or extend `RichParallelSourceFunction` in which cases the source will have the parallelism of the environment. The degree of parallelism for ParallelSourceFunctions can be changed afterwards using `source.setParallelism(dop)`. +To create parallel sources the users source function needs to implement `ParallelSourceFunction` or extend `RichParallelSourceFunction` in which cases the source will have the parallelism of the environment. The parallelism for ParallelSourceFunctions can be changed afterwards using `source.setParallelism(parallelism)`. There are several predefined ones similar to the ones of the batch API and some streaming specific ones like: @@ -751,7 +751,7 @@ The above call would create global windows of 1000 elements group it by the firs Notice that here we only defined the window size once at the beginning of the transformation. This means that anything that happens afterwards (`groupBy(firstKey).mapWindow(…).groupBy(secondKey).reduceWindow(…)`) happens inside the 1000 element windows. Of course the mapWindow might reduce the number of elements but the key idea is that each transformation still corresponds to the same 1000 elements in the original stream. #### Global vs local discretisation -By default all window discretisation calls (`dataStream.window(…)`) define global windows meaning that a global window of count 100 will contain the last 100 elements arrived at the discretisation operator in order. In most cases (except for Time) this means that the operator doing the actual discretisation needs to have a degree of parallelism of 1 to be able to correctly execute the discretisation logic. +By default all window discretisation calls (`dataStream.window(…)`) define global windows meaning that a global window of count 100 will contain the last 100 elements arrived at the discretisation operator in order. In most cases (except for Time) this means that the operator doing the actual discretisation needs to have a parallelism of 1 to be able to correctly execute the discretisation logic. Sometimes it is sufficient to create local discretisations, which allows the discretiser to run in parallel and apply the given discretisation logic at every discretiser instance. To allow local discretisation use the `local()` method of the windowed data stream. @@ -1109,7 +1109,7 @@ Operator Settings ### Parallelism -Setting parallelism for operators works exactly the same way as in the batch Flink API. The user can control the number of parallel instances created for each operator by calling the `operator.setParallelism(dop)` method. +Setting parallelism for operators works exactly the same way as in the batch Flink API. The user can control the number of parallel instances created for each operator by calling the `operator.setParallelism(parallelism)` method. ### Buffer timeout diff --git a/docs/yarn_setup.md b/docs/yarn_setup.md index d65cf7e287350..4da8a057d2f25 100644 --- a/docs/yarn_setup.md +++ b/docs/yarn_setup.md @@ -112,7 +112,7 @@ Please note that the Client requires the `YARN_CONF_DIR` or `HADOOP_CONF_DIR` en The system will use the configuration in `conf/flink-config.yaml`. Please follow our [configuration guide](config.html) if you want to change something. -Flink on YARN will overwrite the following configuration parameters `jobmanager.rpc.address` (because the JobManager is always allocated at different machines), `taskmanager.tmp.dirs` (we are using the tmp directories given by YARN) and `parallelization.degree.default` if the number of slots has been specified. +Flink on YARN will overwrite the following configuration parameters `jobmanager.rpc.address` (because the JobManager is always allocated at different machines), `taskmanager.tmp.dirs` (we are using the tmp directories given by YARN) and `parallelism.default` if the number of slots has been specified. If you don't want to change the configuration file to set configuration parameters, there is the option to pass dynamic properties via the `-D` flag. So you can pass parameters this way: `-Dfs.overwrite-files=true -Dtaskmanager.network.numberOfBuffers=16368`. diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index 1d9d95646f2a3..dd2a0ba011ae1 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -102,7 +102,7 @@ public class CliFrontend { // YARN-session related constants public static final String YARN_PROPERTIES_FILE = ".yarn-properties"; public static final String YARN_PROPERTIES_JOBMANAGER_KEY = "jobManager"; - public static final String YARN_PROPERTIES_DOP = "degreeOfParallelism"; + public static final String YARN_PROPERTIES_PARALLELISM = "parallelism"; public static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString"; public static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split() @@ -175,18 +175,18 @@ public CliFrontend(String configDir) throws Exception { throw new Exception("Cannot read the YARN properties file", e); } - // configure the default degree of parallelism from YARN - String propDegree = yarnProperties.getProperty(YARN_PROPERTIES_DOP); - if (propDegree != null) { // maybe the property is not set + // configure the default parallelism from YARN + String propParallelism = yarnProperties.getProperty(YARN_PROPERTIES_PARALLELISM); + if (propParallelism != null) { // maybe the property is not set try { - int paraDegree = Integer.parseInt(propDegree); - this.config.setInteger(ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY, paraDegree); + int parallelism = Integer.parseInt(propParallelism); + this.config.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism); - logAndSysout("YARN properties set default parallelism to " + paraDegree); + logAndSysout("YARN properties set default parallelism to " + parallelism); } catch (NumberFormatException e) { throw new Exception("Error while parsing the YARN properties: " + - "Property " + YARN_PROPERTIES_DOP + " is not an integer."); + "Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer."); } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java index 340b87864ceb2..95e099b941ac7 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java @@ -410,7 +410,7 @@ public int run(String[] args) { if (flinkYarnClient.getTaskManagerSlots() != -1) { String degreeOfParallelism = Integer.toString(flinkYarnClient.getTaskManagerSlots() * flinkYarnClient.getTaskManagerCount()); - yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_DOP, degreeOfParallelism); + yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_PARALLELISM, degreeOfParallelism); } // add dynamic properties if (flinkYarnClient.getDynamicPropertiesEncoded() != null) { diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 4e593c884c850..9a578bc4b9e23 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -145,7 +145,7 @@ else if (prog.isUsingInteractiveMode()) { // temporary hack to support the optimizer plan preview OptimizerPlanEnvironment env = new OptimizerPlanEnvironment(this.compiler); if (parallelism > 0) { - env.setDegreeOfParallelism(parallelism); + env.setParallelism(parallelism); } env.setAsContext(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java index c1fe9b450e68a..8d5fe17df0a10 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java @@ -52,14 +52,14 @@ public JobExecutionResult execute(String jobName) throws Exception { Plan p = createProgramPlan(jobName); JobWithJars toRun = new JobWithJars(p, this.jarFilesToAttach, this.userCodeClassLoader); - return this.client.run(toRun, getDegreeOfParallelism(), true); + return this.client.run(toRun, getParallelism(), true); } @Override public String getExecutionPlan() throws Exception { Plan p = createProgramPlan("unnamed job"); - OptimizedPlan op = (OptimizedPlan) this.client.getOptimizedPlan(p, getDegreeOfParallelism()); + OptimizedPlan op = (OptimizedPlan) this.client.getOptimizedPlan(p, getParallelism()); PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator(); return gen.getOptimizerPlanAsJSON(op); @@ -68,7 +68,7 @@ public String getExecutionPlan() throws Exception { @Override public String toString() { - return "Context Environment (DOP = " + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + return "Context Environment (parallelism = " + (getParallelism() == -1 ? "default" : getParallelism()) + ") : " + getIdString(); } @@ -118,7 +118,7 @@ public ContextEnvironmentFactory(Client client, List jarFilesToAttach, public ExecutionEnvironment createExecutionEnvironment() { ContextEnvironment env = new ContextEnvironment(client, jarFilesToAttach, userCodeClassLoader); if (defaultParallelism > 0) { - env.setDegreeOfParallelism(defaultParallelism); + env.setParallelism(defaultParallelism); } return env; } 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 a2df4384d956a..1990a2f8ce026 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 @@ -52,7 +52,7 @@ public class ExecutionConfig implements Serializable { public static final String CONFIG_KEY = "runtime.config"; /** - * The constant to use for the degree of parallelism, if the system should use the number + * The constant to use for the parallelism, if the system should use the number * of currently available slots. */ public static final int PARALLELISM_AUTO_MAX = Integer.MAX_VALUE; 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 d5295bb469651..f0cb926e1841d 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 @@ -44,7 +44,7 @@ /** * This class encapsulates a single job (an instantiated data flow), together with some parameters. - * Parameters include the name and a default degree of parallelism. The job is referenced by the data sinks, + * Parameters include the name and a default parallelism. The job is referenced by the data sinks, * from which a traversal reaches all connected nodes of the job. */ public class Plan implements Visitable> { @@ -102,7 +102,7 @@ public Plan(Collection> sinks, String jobName) * * @param sinks The collection will the sinks of the job's data flow. * @param jobName The name to display for the job. - * @param defaultParallelism The default degree of parallelism for the job. + * @param defaultParallelism The default parallelism for the job. */ public Plan(Collection> sinks, String jobName, int defaultParallelism) { this.sinks.addAll(sinks); @@ -134,7 +134,7 @@ public Plan(GenericDataSinkBase sink, String jobName) { * * @param sink The data sink of the data flow. * @param jobName The name to display for the job. - * @param defaultParallelism The default degree of parallelism for the job. + * @param defaultParallelism The default parallelism for the job. */ public Plan(GenericDataSinkBase sink, String jobName, int defaultParallelism) { this(Collections.>singletonList(sink), jobName, defaultParallelism); @@ -163,7 +163,7 @@ public Plan(Collection> sinks) { * from the given data sinks. * * @param sinks The collection will the sinks of the data flow. - * @param defaultParallelism The default degree of parallelism for the job. + * @param defaultParallelism The default parallelism for the job. */ public Plan(Collection> sinks, int defaultParallelism) { this(sinks, "Flink Job at " + Calendar.getInstance().getTime(), defaultParallelism); @@ -190,7 +190,7 @@ public Plan(GenericDataSinkBase sink) { * not be translated entirely. * * @param sink The data sink of the data flow. - * @param defaultParallelism The default degree of parallelism for the job. + * @param defaultParallelism The default parallelism for the job. */ public Plan(GenericDataSinkBase sink, int defaultParallelism) { this(sink, "Flink Job at " + Calendar.getInstance().getTime(), defaultParallelism); @@ -240,8 +240,8 @@ public void setJobName(String jobName) { } /** - * Gets the default degree of parallelism for this job. That degree is always used when an operator - * is not explicitly given a degree of parallelism. + * Gets the default parallelism for this job. That degree is always used when an operator + * is not explicitly given a parallelism. * * @return The default parallelism for the plan. */ @@ -250,14 +250,14 @@ public int getDefaultParallelism() { } /** - * Sets the default degree of parallelism for this plan. That degree is always used when an operator - * is not explicitly given a degree of parallelism. + * Sets the default parallelism for this plan. That degree is always used when an operator + * is not explicitly given a parallelism. * * @param defaultParallelism The default parallelism for the plan. */ public void setDefaultParallelism(int defaultParallelism) { checkArgument(defaultParallelism >= 1 || defaultParallelism == -1, - "The default degree of parallelism must be positive, or -1 if the system should use the globally comfigured default."); + "The default parallelism must be positive, or -1 if the system should use the globally comfigured default."); this.defaultParallelism = defaultParallelism; } @@ -364,7 +364,7 @@ private static final class MaxDopVisitor implements Visitor> { @Override public boolean preVisit(Operator visitable) { - this.maxDop = Math.max(this.maxDop, visitable.getDegreeOfParallelism()); + this.maxDop = Math.max(this.maxDop, visitable.getParallelism()); return true; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index e9209a8aabcd3..f68d2b0a9e045 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -33,7 +33,7 @@ /** * A RuntimeContext contains information about the context in which functions are executed. Each parallel instance * of the function will have a context through which it can access static contextual information (such as - * the current degree of parallelism) and other constructs like accumulators and broadcast variables. + * the current parallelism) and other constructs like accumulators and broadcast variables. *

* A function can, during runtime, obtain the RuntimeContext via a call to * {@link AbstractRichFunction#getRuntimeContext()}. @@ -48,14 +48,14 @@ public interface RuntimeContext { String getTaskName(); /** - * Gets the degree of parallelism with which the parallel task runs. + * Gets the parallelism with which the parallel task runs. * - * @return The degree of parallelism with which the parallel task runs. + * @return The parallelism with which the parallel task runs. */ int getNumberOfParallelSubtasks(); /** - * Gets the number of the parallel subtask. The numbering starts from 1 and goes up to the degree-of-parallelism, + * Gets the number of the parallel subtask. The numbering starts from 1 and goes up to the parallelism, * as returned by {@link #getNumberOfParallelSubtasks()}. * * @return The number of the parallel subtask. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java index 6fa535c4ffe3a..8b787eb50c06e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java @@ -29,7 +29,7 @@ public interface FinalizeOnMaster { /** * The method is invoked on the master (JobManager) after all (parallel) instances of an OutputFormat finished. * - * @param parallelism The degree of parallelism with which the format or functions was run. + * @param parallelism The parallelism with which the format or functions was run. * @throws IOException The finalization may throw exceptions, which may cause the job to abort. */ void finalizeGlobal(int parallelism) throws IOException; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java b/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java index 8d391c02f509b..943fc8d574db7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java @@ -31,7 +31,7 @@ public interface InitializeOnMaster { /** * The method is invoked on the master (JobManager) before the distributed program execution starts. * - * @param parallelism The degree of parallelism with which the format or functions will be run. + * @param parallelism The parallelism with which the format or functions will be run. * @throws IOException The initialization may throw exceptions, which may cause the job to abort. */ void initializeGlobal(int parallelism) throws IOException; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java b/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java index 00a16138befd6..26e49a76077c7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java @@ -20,7 +20,7 @@ /** * This interface acts as a marker for input formats for inputs which cannot be split. - * Data sources with a non-parallel input formats are always executed with a degree-of-parallelism + * Data sources with a non-parallel input formats are always executed with a parallelism * of one. * * @see InputFormat diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java index 0adccafd345a2..cc25539347fb9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java @@ -33,9 +33,9 @@ * replicated InputFormat to each parallel instance. * * Replicated data can only be used as input for a {@link org.apache.flink.api.common.operators.base.JoinOperatorBase} or - * {@link org.apache.flink.api.common.operators.base.CrossOperatorBase} with the same degree of parallelism as the DataSource. + * {@link org.apache.flink.api.common.operators.base.CrossOperatorBase} with the same parallelism as the DataSource. * Before being used as an input to a Join or Cross operator, replicated data might be processed in local pipelines by - * by Map-based operators with the same degree of parallelism as the source. Map-based operators are + * by Map-based operators with the same parallelism as the source. Map-based operators are * {@link org.apache.flink.api.common.operators.base.MapOperatorBase}, * {@link org.apache.flink.api.common.operators.base.FlatMapOperatorBase}, * {@link org.apache.flink.api.common.operators.base.FilterOperatorBase}, and 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 4fc89c38e886a..b472d8afdb2f6 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 @@ -31,9 +31,9 @@ public final class ConfigConstants { // ---------------------------- Parallelism ------------------------------- /** - * The config parameter defining the default degree of parallelism for jobs. + * The config parameter defining the default parallelism for jobs. */ - public static final String DEFAULT_PARALLELIZATION_DEGREE_KEY = "parallelization.degree.default"; + public static final String DEFAULT_PARALLELISM_KEY = "parallelism.default"; /** * Config parameter for the number of re-tries for failed tasks. Setting this @@ -399,9 +399,9 @@ public final class ConfigConstants { // ---------------------------- Parallelism ------------------------------- /** - * The default degree of parallelism for operations. + * The default parallelism for operations. */ - public static final int DEFAULT_PARALLELIZATION_DEGREE = 1; + public static final int DEFAULT_PARALLELISM = 1; /** * The default number of execution retries. diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java index 8c4e0908cf3da..3e4697ab87419 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java @@ -56,7 +56,7 @@ public int compare(FileInputSplit o1, FileInputSplit o2) { protected long blockSize; - private int degreeOfParallelism; + private int parallelism; private int[] rawDataSizes; @@ -65,11 +65,11 @@ public int compare(FileInputSplit o1, FileInputSplit o2) { /** * Initializes SequentialFormatTest. */ - public SequentialFormatTestBase(int numberOfTuples, long blockSize, int degreeOfParallelism) { + public SequentialFormatTestBase(int numberOfTuples, long blockSize, int parallelism) { this.numberOfTuples = numberOfTuples; this.blockSize = blockSize; - this.degreeOfParallelism = degreeOfParallelism; - this.rawDataSizes = new int[degreeOfParallelism]; + this.parallelism = parallelism; + this.rawDataSizes = new int[parallelism]; } /** @@ -78,7 +78,7 @@ public SequentialFormatTestBase(int numberOfTuples, long blockSize, int degreeOf @Before public void calcRawDataSize() throws IOException { int recordIndex = 0; - for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) { + for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { ByteCounter byteCounter = new ByteCounter(); DataOutputStream out = new DataOutputStream(byteCounter); for (int fileCount = 0; fileCount < this.getNumberOfTuplesPerFile(fileIndex); fileCount++, recordIndex++) { @@ -98,7 +98,7 @@ public void checkInputSplits() throws IOException { Arrays.sort(inputSplits, new InputSplitSorter()); int splitIndex = 0; - for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) { + for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { List sameFileSplits = new ArrayList(); Path lastPath = inputSplits[splitIndex].getPath(); for (; splitIndex < inputSplits.length; splitIndex++) { @@ -178,7 +178,7 @@ public void writeTuples() throws IOException { this.tempFile.deleteOnExit(); Configuration configuration = new Configuration(); configuration.setLong(BinaryOutputFormat.BLOCK_SIZE_PARAMETER_KEY, this.blockSize); - if (this.degreeOfParallelism == 1) { + if (this.parallelism == 1) { BinaryOutputFormat output = createOutputFormat(this.tempFile.toURI().toString(), configuration); for (int index = 0; index < this.numberOfTuples; index++) { @@ -189,7 +189,7 @@ public void writeTuples() throws IOException { this.tempFile.delete(); this.tempFile.mkdir(); int recordIndex = 0; - for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) { + for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { BinaryOutputFormat output = createOutputFormat(this.tempFile.toURI() + "/" + (fileIndex+1), configuration); for (int fileCount = 0; fileCount < this.getNumberOfTuplesPerFile(fileIndex); fileCount++, recordIndex++) { @@ -201,7 +201,7 @@ public void writeTuples() throws IOException { } private int getNumberOfTuplesPerFile(int fileIndex) { - return this.numberOfTuples / this.degreeOfParallelism; + return this.numberOfTuples / this.parallelism; } /** @@ -211,7 +211,7 @@ private int getNumberOfTuplesPerFile(int fileIndex) { public void checkLength() { File[] files = this.tempFile.isDirectory() ? this.tempFile.listFiles() : new File[] { this.tempFile }; Arrays.sort(files); - for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) { + for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { long lastBlockLength = this.rawDataSizes[fileIndex] % (this.blockSize - getInfoSize()); long expectedLength = (this.getExpectedBlockCount(fileIndex) - 1) * this.blockSize + getInfoSize() + @@ -252,13 +252,13 @@ private int getExpectedBlockCount(int fileIndex) { @Parameters public static List getParameters() { ArrayList params = new ArrayList(); - for (int dop = 1; dop <= 2; dop++) { - // numberOfTuples, blockSize, dop - params.add(new Object[] { 100, BinaryOutputFormat.NATIVE_BLOCK_SIZE, dop }); - params.add(new Object[] { 100, 1000, dop }); - params.add(new Object[] { 100, 1 << 20, dop }); - params.add(new Object[] { 10000, 1000, dop }); - params.add(new Object[] { 10000, 1 << 20, dop }); + for (int parallelism = 1; parallelism <= 2; parallelism++) { + // numberOfTuples, blockSize, parallelism + params.add(new Object[] { 100, BinaryOutputFormat.NATIVE_BLOCK_SIZE, parallelism }); + params.add(new Object[] { 100, 1000, parallelism }); + params.add(new Object[] { 100, 1 << 20, parallelism }); + params.add(new Object[] { 10000, 1000, parallelism }); + params.add(new Object[] { 10000, 1 << 20, parallelism }); } return params; } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java index 90347b87c8891..e421f4f7b8c07 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java @@ -35,8 +35,8 @@ public class SerializedFormatTest extends SequentialFormatTestBase { private BlockInfo info; - public SerializedFormatTest(int numberOfRecords, long blockSize, int degreeOfParallelism){ - super(numberOfRecords, blockSize, degreeOfParallelism); + public SerializedFormatTest(int numberOfRecords, long blockSize, int parallelism){ + super(numberOfRecords, blockSize, parallelism); } @Before diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index 894137f48ff55..2f2eb225be20d 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -31,7 +31,7 @@ taskmanager.heap.mb: 512 taskmanager.numberOfTaskSlots: 1 -parallelization.degree.default: 1 +parallelism.default: 1 #============================================================================== # Web Frontend diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index b045100dd788a..ed8d1caab8305 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -222,7 +222,7 @@ public MapOperator map(MapFunction mapper) { * The function is called once per parallel partition of the data, * and the entire partition is available through the given Iterator. * The number of elements that each instance of the MapPartition function - * sees is non deterministic and depends on the degree of parallelism of the operation. + * sees is non deterministic and depends on the parallelism of the operation. * * This function is intended for operations that cannot transform individual elements, * requires no grouping of elements. To transform individual elements, diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index aac3147f8f6a5..9d1fc3662cacf 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -133,14 +133,14 @@ public ExecutionConfig getConfig() { } /** - * Gets the degree of parallelism with which operation are executed by default. Operations can - * individually override this value to use a specific degree of parallelism via + * Gets the parallelism with which operation are executed by default. Operations can + * individually override this value to use a specific parallelism via * {@link Operator#setParallelism(int)}. Other operations may need to run with a different - * degree of parallelism - for example calling + * parallelism - for example calling * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire - * set will insert eventually an operation that runs non-parallel (degree of parallelism of one). + * set will insert eventually an operation that runs non-parallel (parallelism of one). * - * @return The degree of parallelism used by operations, unless they override that value. This method + * @return The parallelism used by operations, unless they override that value. This method * returns {@code -1}, if the environments default parallelism should be used. * @deprecated Please use {@link #getParallelism} */ @@ -165,14 +165,14 @@ public int getParallelism() { } /** - * Sets the degree of parallelism (DOP) for operations executed through this environment. - * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with + * Sets the parallelism for operations executed through this environment. + * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with * x parallel instances. *

* This method overrides the default parallelism for this environment. * The {@link LocalEnvironment} uses by default a value equal to the number of hardware * contexts (CPU cores / threads). When executing the program via the command line client - * from a JAR file, the default degree of parallelism is the one configured for that setup. + * from a JAR file, the default parallelism is the one configured for that setup. * * @param parallelism The parallelism * @deprecated Please use {@link #setParallelism} @@ -592,7 +592,7 @@ public DataSource> createHadoopInput(org.apache.hadoop.mapred * via {@link #fromCollection(Collection, TypeInformation)}. *

* Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. * * @param data The collection of elements to create the data set from. * @return A DataSet representing the given collection. @@ -621,7 +621,7 @@ public DataSource fromCollection(Collection data) { * if needed. *

* Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. *

* The returned DataSet is typed to the given TypeInformation. * @@ -651,7 +651,7 @@ private DataSource fromCollection(Collection data, TypeInformation * framework may move it to a remote environment, if needed. *

* Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. * * @param data The collection of elements to create the data set from. * @param type The class of the data produced by the iterator. Must not be a generic class. @@ -674,7 +674,7 @@ public DataSource fromCollection(Iterator data, Class type) { * framework may move it to a remote environment, if needed. *

* Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. * * @param data The collection of elements to create the data set from. * @param type The TypeInformation for the produced data set. @@ -702,7 +702,7 @@ public DataSource fromCollection(Iterator data, TypeInformation typ * via {@link #fromCollection(Collection, TypeInformation)}. *

* Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. * * @param data The elements to make up the data set. * @return A DataSet representing the given list of elements. @@ -924,8 +924,8 @@ public JavaPlan createProgramPlan(String jobName, boolean clearSinks) { OperatorTranslation translator = new OperatorTranslation(); JavaPlan plan = translator.translateToPlan(this.sinks, jobName); - if (getDegreeOfParallelism() > 0) { - plan.setDefaultParallelism(getDegreeOfParallelism()); + if (getParallelism() > 0) { + plan.setDefaultParallelism(getParallelism()); } plan.setExecutionConfig(getConfig()); // Check plan for GenericTypeInfo's and register the types at the serializers. @@ -1024,18 +1024,18 @@ public static ExecutionEnvironment getExecutionEnvironment() { /** * Creates a {@link CollectionEnvironment} that uses Java Collections underneath. This will execute in a * single thread in the current JVM. It is very fast but will fail if the data does not fit into - * memory. Degree of parallelism will always be 1. This is useful during implementation and for debugging. + * memory. parallelism will always be 1. This is useful during implementation and for debugging. * @return A Collection Environment */ public static CollectionEnvironment createCollectionsEnvironment(){ CollectionEnvironment ce = new CollectionEnvironment(); - ce.setDegreeOfParallelism(1); + ce.setParallelism(1); return ce; } /** * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a - * multi-threaded fashion in the same JVM as the environment was created in. The default degree of + * multi-threaded fashion in the same JVM as the environment was created in. The default * parallelism of the local environment is the number of hardware contexts (CPU cores / threads), * unless it was specified differently by {@link #setDefaultLocalParallelism(int)}. * @@ -1048,22 +1048,22 @@ public static LocalEnvironment createLocalEnvironment() { /** * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a * multi-threaded fashion in the same JVM as the environment was created in. It will use the - * degree of parallelism specified in the parameter. + * parallelism specified in the parameter. * - * @param degreeOfParallelism The degree of parallelism for the local environment. - * @return A local execution environment with the specified degree of parallelism. + * @param parallelism The parallelism for the local environment. + * @return A local execution environment with the specified parallelism. */ - public static LocalEnvironment createLocalEnvironment(int degreeOfParallelism) { + public static LocalEnvironment createLocalEnvironment(int parallelism) { LocalEnvironment lee = new LocalEnvironment(); - lee.setDegreeOfParallelism(degreeOfParallelism); + lee.setParallelism(parallelism); return lee; } /** * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The execution will use the cluster's default degree of parallelism, unless the parallelism is - * set explicitly via {@link ExecutionEnvironment#setDegreeOfParallelism(int)}. + * cluster. The execution will use the cluster's default parallelism, unless the parallelism is + * set explicitly via {@link ExecutionEnvironment#setParallelism(int)}. * * @param host The host name or address of the master (JobManager), where the program should be executed. * @param port The port of the master (JobManager), where the program should be executed. @@ -1079,19 +1079,19 @@ public static ExecutionEnvironment createRemoteEnvironment(String host, int port /** * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The execution will use the specified degree of parallelism. + * cluster. The execution will use the specified parallelism. * * @param host The host name or address of the master (JobManager), where the program should be executed. * @param port The port of the master (JobManager), where the program should be executed. - * @param degreeOfParallelism The degree of parallelism to use during the execution. + * @param parallelism The parallelism to use during the execution. * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses * user-defined functions, user-defined input formats, or any libraries, those must be * provided in the JAR files. * @return A remote environment that executes the program on a cluster. */ - public static ExecutionEnvironment createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles) { + public static ExecutionEnvironment createRemoteEnvironment(String host, int port, int parallelism, String... jarFiles) { RemoteEnvironment rec = new RemoteEnvironment(host, port, jarFiles); - rec.setDegreeOfParallelism(degreeOfParallelism); + rec.setParallelism(parallelism); return rec; } @@ -1099,10 +1099,10 @@ public static ExecutionEnvironment createRemoteEnvironment(String host, int port * Sets the default parallelism that will be used for the local execution environment created by * {@link #createLocalEnvironment()}. * - * @param degreeOfParallelism The degree of parallelism to use as the default local parallelism. + * @param parallelism The parallelism to use as the default local parallelism. */ - public static void setDefaultLocalParallelism(int degreeOfParallelism) { - defaultLocalDop = degreeOfParallelism; + public static void setDefaultLocalParallelism(int parallelism) { + defaultLocalDop = parallelism; } // -------------------------------------------------------------------------------------------- diff --git a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java index e7daf11e21bad..beecaf6034787 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java @@ -25,10 +25,10 @@ /** * An {@link ExecutionEnvironment} that runs the program locally, multi-threaded, in the JVM where the - * environment is instantiated. When this environment is instantiated, it uses a default degree of parallelism + * environment is instantiated. When this environment is instantiated, it uses a default parallelism * of {@code 1}. Local environments can also be instantiated through * {@link ExecutionEnvironment#createLocalEnvironment()} and {@link ExecutionEnvironment#createLocalEnvironment(int)}. - * The former version will pick a default degree of parallelism equal to the number of hardware contexts in the local + * The former version will pick a default parallelism equal to the number of hardware contexts in the local * machine. */ public class LocalEnvironment extends ExecutionEnvironment { @@ -62,7 +62,7 @@ public String getExecutionPlan() throws Exception { @Override public String toString() { - return "Local Environment (DOP = " + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + return "Local Environment (parallelism = " + (getParallelism() == -1 ? "default" : getParallelism()) + ") : " + getIdString(); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java index c0695e52e72b9..df18bbfa61527 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java @@ -25,8 +25,8 @@ /** * An {@link ExecutionEnvironment} that sends programs * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The execution will use the cluster's default degree of parallelism, unless the parallelism is - * set explicitly via {@link ExecutionEnvironment#setDegreeOfParallelism(int)}. + * cluster. The execution will use the cluster's default parallelism, unless the parallelism is + * set explicitly via {@link ExecutionEnvironment#setParallelism(int)}. */ public class RemoteEnvironment extends ExecutionEnvironment { @@ -72,7 +72,7 @@ public JobExecutionResult execute(String jobName) throws Exception { @Override public String getExecutionPlan() throws Exception { Plan p = createProgramPlan("unnamed", false); - p.setDefaultParallelism(getDegreeOfParallelism()); + p.setDefaultParallelism(getParallelism()); registerCachedFilesWithPlan(p); PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, jarFiles); @@ -81,7 +81,7 @@ public String getExecutionPlan() throws Exception { @Override public String toString() { - return "Remote Environment (" + this.host + ":" + this.port + " - DOP = " + - (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + ") : " + getIdString(); + return "Remote Environment (" + this.host + ":" + this.port + " - parallelism = " + + (getParallelism() == -1 ? "default" : getParallelism()) + ") : " + getIdString(); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java index 7c09518094432..c8f8bbc0b8443 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java @@ -169,10 +169,10 @@ private void appendSorting(int[] flatOrderFields, Order order) { noop.setInput(input); if(this.getParallelism() < 0) { // use parallelism of input if not explicitly specified - noop.setDegreeOfParallelism(input.getDegreeOfParallelism()); + noop.setParallelism(input.getParallelism()); } else { // use explicitly specified parallelism - noop.setDegreeOfParallelism(this.getParallelism()); + noop.setParallelism(this.getParallelism()); } return noop; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java index 41410da72b408..8e028eb963653 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java @@ -73,7 +73,7 @@ public void open(int taskNumber, int numTasks) { // get the target stream this.stream = this.target == STD_OUT ? System.out : System.err; - // set the prefix if we have a >1 DOP + // set the prefix if we have a >1 parallelism this.prefix = (numTasks > 1) ? ((taskNumber+1) + "> ") : null; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java index 66821ae6d106d..daaedacf6701f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java @@ -193,8 +193,8 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase getPartitioner() { po = translateSelectorFunctionCoGroup(selectorKeys1, selectorKeys2, function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - po.setDegreeOfParallelism(getParallelism()); + po.setParallelism(getParallelism()); po.setCustomPartitioner(customPartitioner); } else if (keys2 instanceof Keys.SelectorFunctionKeys) { @@ -204,7 +204,7 @@ else if (keys2 instanceof Keys.SelectorFunctionKeys) { po = translateSelectorFunctionCoGroupRight(logicalKeyPositions1, selectorKeys2, function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - po.setDegreeOfParallelism(getParallelism()); + po.setParallelism(getParallelism()); po.setCustomPartitioner(customPartitioner); } else if (keys1 instanceof Keys.SelectorFunctionKeys) { @@ -242,7 +242,7 @@ else if ( keys1 instanceof Keys.ExpressionKeys && keys2 instanceof Keys.Expressi } // configure shared characteristics - po.setDegreeOfParallelism(getParallelism()); + po.setParallelism(getParallelism()); po.setCustomPartitioner(customPartitioner); if (groupSortKeyOrderFirst.size() > 0) { @@ -292,9 +292,9 @@ private static PlanBothUnwrappingCoGroupOperator PlanRightUnwrappingCoGroupOperator PlanLeftUnwrappingCoGroupOperator { private String name; - private int dop = -1; + private int parallelism = -1; private Configuration parameters; @@ -234,13 +234,13 @@ protected GenericDataSinkBase translateToDataFlow(Operator input) { if(this.parameters != null) { sink.getParameters().addAll(this.parameters); } - // set dop - if(this.dop > 0) { - // use specified dop - sink.setDegreeOfParallelism(this.dop); + // set parallelism + if(this.parallelism > 0) { + // use specified parallelism + sink.setParallelism(this.parallelism); } else { - // if no dop has been specified, use dop of input operator to enable chaining - sink.setDegreeOfParallelism(input.getDegreeOfParallelism()); + // if no parallelism has been specified, use parallelism of input operator to enable chaining + sink.setParallelism(input.getParallelism()); } if(this.sortKeyPositions != null) { @@ -263,27 +263,27 @@ public String toString() { } /** - * Returns the degree of parallelism of this data sink. + * Returns the parallelism of this data sink. * - * @return The degree of parallelism of this data sink. + * @return The parallelism of this data sink. */ public int getParallelism() { - return this.dop; + return this.parallelism; } /** - * Sets the degree of parallelism for this data sink. + * Sets the parallelism for this data sink. * The degree must be 1 or more. * - * @param dop The degree of parallelism for this data sink. - * @return This data sink with set degree of parallelism. + * @param parallelism The parallelism for this data sink. + * @return This data sink with set parallelism. */ - public DataSink setParallelism(int dop) { + public DataSink setParallelism(int parallelism) { - if(dop < 1) { + if(parallelism < 1) { throw new IllegalArgumentException("The parallelism of an operator must be at least 1."); } - this.dop = dop; + this.parallelism = parallelism; return this; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java index d6e511af843da..728985a7d1595 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java @@ -65,7 +65,7 @@ public DataSource(ExecutionEnvironment context, InputFormat inputFormat, this.inputFormat = inputFormat; if (inputFormat instanceof NonParallelInput) { - this.dop = 1; + this.parallelism = 1; } } @@ -126,7 +126,7 @@ public SplitDataProperties getSplitDataProperties() { @SuppressWarnings({ "unchecked", "rawtypes" }) GenericDataSourceBase source = new GenericDataSourceBase(this.inputFormat, new OperatorInformation(getType()), name); - source.setDegreeOfParallelism(dop); + source.setParallelism(parallelism); if(this.parameters != null) { source.getParameters().addAll(this.parameters); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java index aec495097d114..4d61aec380c36 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java @@ -165,19 +165,19 @@ public String getName() { } /** - * Sets the degree of parallelism for the iteration. + * Sets the parallelism for the iteration. * - * @param parallelism The degree of parallelism. + * @param parallelism The parallelism. * @return The iteration object, for function call chaining. */ public DeltaIteration parallelism(int parallelism) { - Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default)."); + Validate.isTrue(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default)."); this.parallelism = parallelism; return this; } /** - * Gets the iteration's degree of parallelism. + * Gets the iteration's parallelism. * * @return The iterations parallelism, or -1, if not set. */ diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java index d570fc26d8740..686823cf6792d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java @@ -88,7 +88,7 @@ public DistinctOperator(DataSet input, Keys keys, String distinctLocationN po.setCombinable(true); po.setInput(input); - po.setDegreeOfParallelism(getParallelism()); + po.setParallelism(getParallelism()); // make sure that distinct preserves the partitioning for the fields on which they operate if (getType().isTupleType()) { @@ -113,7 +113,7 @@ else if (keys instanceof Keys.SelectorFunctionKeys) { PlanUnwrappingReduceGroupOperator po = translateSelectorFunctionDistinct( selectorKeys, function, getInputType(), getResultType(), name, input); - po.setDegreeOfParallelism(this.getParallelism()); + po.setParallelism(this.getParallelism()); return po; } @@ -145,7 +145,7 @@ private static PlanUnwrappingReduceGroupOperator transl mapper.setInput(input); // set the mapper's parallelism to the input parallelism to make sure it is chained - mapper.setDegreeOfParallelism(input.getDegreeOfParallelism()); + mapper.setParallelism(input.getParallelism()); return reducer; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java index 56bea50b16580..f55de1c42b3b3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java @@ -57,13 +57,13 @@ protected org.apache.flink.api.common.operators.base.FilterOperatorBase po = new PlanFilterOperator(function, name, getInputType()); po.setInput(input); - // set dop + // set parallelism if (getParallelism() > 0) { - // use specified dop - po.setDegreeOfParallelism(getParallelism()); + // use specified parallelism + po.setParallelism(getParallelism()); } else { - // if no dop has been specified, use dop of input operator to enable chaining - po.setDegreeOfParallelism(input.getDegreeOfParallelism()); + // if no parallelism has been specified, use parallelism of input operator to enable chaining + po.setParallelism(input.getParallelism()); } return po; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java index 47446dd1f74bd..55cbb0fbde2a0 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java @@ -57,13 +57,13 @@ protected FlatMapOperatorBase> translateToDataF FlatMapOperatorBase> po = new FlatMapOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); - // set dop + // set parallelism if(this.getParallelism() > 0) { - // use specified dop - po.setDegreeOfParallelism(this.getParallelism()); + // use specified parallelism + po.setParallelism(this.getParallelism()); } else { - // if no dop has been specified, use dop of input operator to enable chaining - po.setDegreeOfParallelism(input.getDegreeOfParallelism()); + // if no parallelism has been specified, use parallelism of input operator to enable chaining + po.setParallelism(input.getParallelism()); } return po; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java index 617162b75c290..3c1d47cdf3d29 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java @@ -103,8 +103,8 @@ protected FlatCombineFunction getFunction() { new GroupCombineOperatorBase>(function, operatorInfo, new int[0], name); po.setInput(input); - // the degree of parallelism for a non grouped reduce can only be 1 - po.setDegreeOfParallelism(1); + // the parallelism for a non grouped reduce can only be 1 + po.setParallelism(1); return po; } @@ -130,13 +130,13 @@ protected FlatCombineFunction getFunction() { } po.setGroupOrder(o); - po.setDegreeOfParallelism(this.getParallelism()); + po.setParallelism(this.getParallelism()); return po; } else { PlanUnwrappingGroupCombineOperator po = translateSelectorFunctionReducer( selectorKeys, function, getInputType(), getResultType(), name, input); - po.setDegreeOfParallelism(this.getParallelism()); + po.setParallelism(this.getParallelism()); return po; } } @@ -148,7 +148,7 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { new GroupCombineOperatorBase>(function, operatorInfo, logicalKeyPositions, name); po.setInput(input); - po.setDegreeOfParallelism(getParallelism()); + po.setParallelism(getParallelism()); // set group order if (grouper instanceof SortedGrouping) { @@ -193,7 +193,7 @@ private static PlanUnwrappingGroupCombineOperator trans mapper.setInput(input); // set the mapper's parallelism to the input parallelism to make sure it is chained - mapper.setDegreeOfParallelism(input.getDegreeOfParallelism()); + mapper.setParallelism(input.getParallelism()); return reducer; } @@ -220,7 +220,7 @@ private static PlanUnwrappingSortedGroupCombineOperator setCombinable(boolean combinable) { po.setCombinable(combinable); po.setInput(input); - // the degree of parallelism for a non grouped reduce can only be 1 - po.setDegreeOfParallelism(1); + // the parallelism for a non grouped reduce can only be 1 + po.setParallelism(1); return po; } @@ -165,14 +165,14 @@ public GroupReduceOperator setCombinable(boolean combinable) { } po.setGroupOrder(o); - po.setDegreeOfParallelism(this.getParallelism()); + po.setParallelism(this.getParallelism()); po.setCustomPartitioner(grouper.getCustomPartitioner()); return po; } else { PlanUnwrappingReduceGroupOperator po = translateSelectorFunctionReducer( selectorKeys, function, getInputType(), getResultType(), name, input, isCombinable()); - po.setDegreeOfParallelism(this.getParallelism()); + po.setParallelism(this.getParallelism()); po.setCustomPartitioner(grouper.getCustomPartitioner()); return po; } @@ -186,7 +186,7 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { po.setCombinable(combinable); po.setInput(input); - po.setDegreeOfParallelism(getParallelism()); + po.setParallelism(getParallelism()); po.setCustomPartitioner(grouper.getCustomPartitioner()); // set group order @@ -233,7 +233,7 @@ private static PlanUnwrappingReduceGroupOperator transl mapper.setInput(input); // set the mapper's parallelism to the input parallelism to make sure it is chained - mapper.setDegreeOfParallelism(input.getDegreeOfParallelism()); + mapper.setParallelism(input.getParallelism()); return reducer; } @@ -261,7 +261,7 @@ private static PlanUnwrappingSortedReduceGroupOperator udfClass) { translateSelectorFunctionJoin(selectorKeys1, selectorKeys2, function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - // set dop - po.setDegreeOfParallelism(this.getParallelism()); + // set parallelism + po.setParallelism(this.getParallelism()); translated = po; } @@ -292,8 +292,8 @@ else if (keys2 instanceof Keys.SelectorFunctionKeys) { function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - // set dop - po.setDegreeOfParallelism(this.getParallelism()); + // set parallelism + po.setParallelism(this.getParallelism()); translated = po; } @@ -311,8 +311,8 @@ else if (keys1 instanceof Keys.SelectorFunctionKeys) { translateSelectorFunctionJoinLeft(selectorKeys1, logicalKeyPositions2, function, getInput1Type(), getInput2Type(), getResultType(), name, input1, input2); - // set dop - po.setDegreeOfParallelism(this.getParallelism()); + // set parallelism + po.setParallelism(this.getParallelism()); translated = po; } @@ -332,8 +332,8 @@ else if (super.keys1 instanceof Keys.ExpressionKeys && super.keys2 instanceof Ke // set inputs po.setFirstInput(input1); po.setSecondInput(input2); - // set dop - po.setDegreeOfParallelism(this.getParallelism()); + // set parallelism + po.setParallelism(this.getParallelism()); translated = po; } @@ -375,9 +375,9 @@ private static PlanBothUnwrappingJoinOperator t keyMapper1.setInput(input1); keyMapper2.setInput(input2); - // set dop - keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism()); - keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism()); + // set parallelism + keyMapper1.setParallelism(input1.getParallelism()); + keyMapper2.setParallelism(input2.getParallelism()); return join; } @@ -427,8 +427,8 @@ private static PlanRightUnwrappingJoinOperator join.setSecondInput(keyMapper2); keyMapper2.setInput(input2); - // set dop - keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism()); + // set parallelism + keyMapper2.setParallelism(input2.getParallelism()); return join; } @@ -477,8 +477,8 @@ private static PlanLeftUnwrappingJoinOperator t join.setSecondInput(input2); keyMapper1.setInput(input1); - // set dop - keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism()); + // set parallelism + keyMapper1.setParallelism(input1.getParallelism()); return join; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java index 7d2bbaaf12466..2663a2a9be428 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java @@ -61,13 +61,13 @@ protected MapOperatorBase> translateToDataFlow(Ope new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); - // set dop + // set parallelism if(this.getParallelism() > 0) { - // use specified dop - po.setDegreeOfParallelism(this.getParallelism()); + // use specified parallelism + po.setParallelism(this.getParallelism()); } else { - // if no dop has been specified, use dop of input operator to enable chaining - po.setDegreeOfParallelism(input.getDegreeOfParallelism()); + // if no parallelism has been specified, use parallelism of input operator to enable chaining + po.setParallelism(input.getParallelism()); } return po; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java index a6c69c10c3987..d8a1abd14e089 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java @@ -60,13 +60,13 @@ protected MapPartitionOperatorBase> trans MapPartitionOperatorBase> po = new MapPartitionOperatorBase>(function, new UnaryOperatorInformation(getInputType(), getResultType()), name); // set input po.setInput(input); - // set dop + // set parallelism if(this.getParallelism() > 0) { - // use specified dop - po.setDegreeOfParallelism(this.getParallelism()); + // use specified parallelism + po.setParallelism(this.getParallelism()); } else { - // if no dop has been specified, use dop of input operator to enable chaining - po.setDegreeOfParallelism(input.getDegreeOfParallelism()); + // if no parallelism has been specified, use parallelism of input operator to enable chaining + po.setParallelism(input.getParallelism()); } return po; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java index 0f8a3eb7f7683..6d0274982c549 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java @@ -32,7 +32,7 @@ public abstract class Operator> extends DataSet< protected String name; - protected int dop = -1; + protected int parallelism = -1; protected Operator(ExecutionEnvironment context, TypeInformation resultType) { super(context, resultType); @@ -58,12 +58,12 @@ public String getName() { } /** - * Returns the degree of parallelism of this operator. + * Returns the parallelism of this operator. * - * @return The degree of parallelism of this operator. + * @return The parallelism of this operator. */ public int getParallelism() { - return this.dop; + return this.parallelism; } /** @@ -82,17 +82,17 @@ public O name(String newName) { } /** - * Sets the degree of parallelism for this operator. + * Sets the parallelism for this operator. * The degree must be 1 or more. * - * @param dop The degree of parallelism for this operator. - * @return The operator with set degree of parallelism. + * @param parallelism The parallelism for this operator. + * @return The operator with set parallelism. */ - public O setParallelism(int dop) { - if(dop < 1) { + public O setParallelism(int parallelism) { + if(parallelism < 1) { throw new IllegalArgumentException("The parallelism of an operator must be at least 1."); } - this.dop = dop; + this.parallelism = parallelism; @SuppressWarnings("unchecked") O returnType = (O) this; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java index 68a216baf82e5..28c1c299b754e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java @@ -218,7 +218,7 @@ private DeltaIterationBase translateDeltaIteration(DeltaIterationRe iterationOperator.setMaximumNumberOfIterations(iterationEnd.getMaxIterations()); if (iterationHead.getParallelism() > 0) { - iterationOperator.setDegreeOfParallelism(iterationHead.getParallelism()); + iterationOperator.setParallelism(iterationHead.getParallelism()); } DeltaIteration.SolutionSetPlaceHolder solutionSetPlaceHolder = iterationHead.getSolutionSet(); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java index edb5a68a224c6..bf9c8e8011995 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java @@ -118,7 +118,7 @@ public Partitioner getCustomPartitioner() { PartitionOperatorBase noop = new PartitionOperatorBase(operatorInfo, pMethod, name); noop.setInput(input); - noop.setDegreeOfParallelism(getParallelism()); + noop.setParallelism(getParallelism()); return noop; } @@ -131,7 +131,7 @@ else if (pMethod == PartitionMethod.HASH || pMethod == PartitionMethod.CUSTOM) { PartitionOperatorBase noop = new PartitionOperatorBase(operatorInfo, pMethod, logicalKeyPositions, name); noop.setInput(input); - noop.setDegreeOfParallelism(getParallelism()); + noop.setParallelism(getParallelism()); noop.setCustomPartitioner(customPartitioner); return noop; @@ -177,10 +177,10 @@ else if (pMethod == PartitionMethod.RANGE) { noop.setCustomPartitioner(customPartitioner); - // set dop - keyExtractingMap.setDegreeOfParallelism(input.getDegreeOfParallelism()); - noop.setDegreeOfParallelism(partitionDop); - keyRemovingMap.setDegreeOfParallelism(partitionDop); + // set parallelism + keyExtractingMap.setParallelism(input.getParallelism()); + noop.setParallelism(partitionDop); + keyRemovingMap.setParallelism(partitionDop); return keyRemovingMap; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java index 16d9ff334fa70..9b7d5674d4a26 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java @@ -71,8 +71,8 @@ protected org.apache.flink.api.common.operators.base.MapOperatorBase ppo = new PlanProjectOperator(fields, name, getInputType(), getResultType(), context.getConfig()); // set input ppo.setInput(input); - // set dop - ppo.setDegreeOfParallelism(this.getParallelism()); + // set parallelism + ppo.setParallelism(this.getParallelism()); ppo.setSemanticProperties(SemanticPropUtil.createProjectionPropertiesSingle(fields, (CompositeType) getInputType())); return ppo; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java index d1ad4c397ee10..5951df832c70d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java @@ -90,8 +90,8 @@ protected ReduceFunction getFunction() { new ReduceOperatorBase>(function, operatorInfo, new int[0], name); po.setInput(input); - // the degree of parallelism for a non grouped reduce can only be 1 - po.setDegreeOfParallelism(1); + // the parallelism for a non grouped reduce can only be 1 + po.setParallelism(1); return po; } @@ -118,7 +118,7 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { po.setCustomPartitioner(grouper.getCustomPartitioner()); po.setInput(input); - po.setDegreeOfParallelism(getParallelism()); + po.setParallelism(getParallelism()); return po; } @@ -130,7 +130,7 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { // -------------------------------------------------------------------------------------------- private static MapOperatorBase, T, ?> translateSelectorFunctionReducer(Keys.SelectorFunctionKeys rawKeys, - ReduceFunction function, TypeInformation inputType, String name, Operator input, int dop) + ReduceFunction function, TypeInformation inputType, String name, Operator input, int parallelism) { @SuppressWarnings("unchecked") final Keys.SelectorFunctionKeys keys = (Keys.SelectorFunctionKeys) rawKeys; @@ -148,10 +148,10 @@ else if (grouper.getKeys() instanceof Keys.ExpressionKeys) { reducer.setInput(keyExtractingMap); keyRemovingMap.setInput(reducer); - // set dop - keyExtractingMap.setDegreeOfParallelism(input.getDegreeOfParallelism()); - reducer.setDegreeOfParallelism(dop); - keyRemovingMap.setDegreeOfParallelism(dop); + // set parallelism + keyExtractingMap.setParallelism(input.getParallelism()); + reducer.setParallelism(parallelism); + keyRemovingMap.setParallelism(parallelism); return keyRemovingMap; } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java index f6e3c2a4a3b69..35c564b9e34b7 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java @@ -47,8 +47,8 @@ public class TypeSerializerFormatTest extends SequentialFormatTestBase> initialData = @@ -58,7 +58,7 @@ public void translateAggregate() { assertEquals(1, reducer.getKeyColumns(0).length); assertEquals(0, reducer.getKeyColumns(0)[0]); - assertEquals(-1, reducer.getDegreeOfParallelism()); + assertEquals(-1, reducer.getParallelism()); assertTrue(reducer.isCombinable()); assertTrue(reducer.getInput() instanceof GenericDataSourceBase); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java index 55a2affc280c1..ae89780c33b28 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java @@ -59,14 +59,14 @@ public void testCorrectTranslation() { final int[] ITERATION_KEYS = new int[] {2}; final int NUM_ITERATIONS = 13; - final int DEFAULT_DOP= 133; - final int ITERATION_DOP = 77; + final int DEFAULT_parallelism= 133; + final int ITERATION_parallelism = 77; ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); // ------------ construct the test program ------------------ { - env.setDegreeOfParallelism(DEFAULT_DOP); + env.setParallelism(DEFAULT_parallelism); @SuppressWarnings("unchecked") DataSet> initialSolutionSet = env.fromElements(new Tuple3(3.44, 5L, "abc")); @@ -75,7 +75,7 @@ public void testCorrectTranslation() { DataSet> initialWorkSet = env.fromElements(new Tuple2(1.23, "abc")); DeltaIteration, Tuple2> iteration = initialSolutionSet.iterateDelta(initialWorkSet, NUM_ITERATIONS, ITERATION_KEYS); - iteration.name(ITERATION_NAME).parallelism(ITERATION_DOP); + iteration.name(ITERATION_NAME).parallelism(ITERATION_parallelism); iteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator()); @@ -100,7 +100,7 @@ public void testCorrectTranslation() { // ------------- validate the plan ---------------- assertEquals(JOB_NAME, p.getJobName()); - assertEquals(DEFAULT_DOP, p.getDefaultParallelism()); + assertEquals(DEFAULT_parallelism, p.getDefaultParallelism()); // validate the iteration GenericDataSinkBase sink1, sink2; @@ -118,7 +118,7 @@ public void testCorrectTranslation() { // check the basic iteration properties assertEquals(NUM_ITERATIONS, iteration.getMaximumNumberOfIterations()); assertArrayEquals(ITERATION_KEYS, iteration.getSolutionSetKeyFields()); - assertEquals(ITERATION_DOP, iteration.getDegreeOfParallelism()); + assertEquals(ITERATION_parallelism, iteration.getParallelism()); assertEquals(ITERATION_NAME, iteration.getName()); MapOperatorBase nextWorksetMapper = (MapOperatorBase) iteration.getNextWorkset(); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java index 0cf2ee2fe3736..b7fbb78806a28 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java @@ -76,8 +76,8 @@ public void testCombinable() { @Test public void translateDistinctPlain() { try { - final int DOP = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP); + final int parallelism = 8; + ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); DataSet> initialData = getSourceDataSet(env); @@ -97,8 +97,8 @@ public void translateDistinctPlain() { // check keys assertArrayEquals(new int[] {0, 1, 2}, reducer.getKeyColumns(0)); - // DOP was not configured on the operator - assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1); + // parallelism was not configured on the operator + assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1); assertTrue(reducer.getInput() instanceof GenericDataSourceBase); } @@ -112,8 +112,8 @@ public void translateDistinctPlain() { @Test public void translateDistinctPlain2() { try { - final int DOP = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP); + final int parallelism = 8; + ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); DataSet initialData = getSourcePojoDataSet(env); @@ -133,8 +133,8 @@ public void translateDistinctPlain2() { // check keys assertArrayEquals(new int[] {0}, reducer.getKeyColumns(0)); - // DOP was not configured on the operator - assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1); + // parallelism was not configured on the operator + assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1); assertTrue(reducer.getInput() instanceof GenericDataSourceBase); } @@ -148,8 +148,8 @@ public void translateDistinctPlain2() { @Test public void translateDistinctPosition() { try { - final int DOP = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP); + final int parallelism = 8; + ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); DataSet> initialData = getSourceDataSet(env); @@ -169,8 +169,8 @@ public void translateDistinctPosition() { // check keys assertArrayEquals(new int[] {1, 2}, reducer.getKeyColumns(0)); - // DOP was not configured on the operator - assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1); + // parallelism was not configured on the operator + assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1); assertTrue(reducer.getInput() instanceof GenericDataSourceBase); } @@ -184,8 +184,8 @@ public void translateDistinctPosition() { @Test public void translateDistinctKeySelector() { try { - final int DOP = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP); + final int parallelism = 8; + ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); DataSet> initialData = getSourceDataSet(env); @@ -202,9 +202,9 @@ public StringValue getKey(Tuple3 value) { PlanUnwrappingReduceGroupOperator reducer = (PlanUnwrappingReduceGroupOperator) sink.getInput(); MapOperatorBase keyExtractor = (MapOperatorBase) reducer.getInput(); - // check the DOPs - assertEquals(1, keyExtractor.getDegreeOfParallelism()); - assertEquals(4, reducer.getDegreeOfParallelism()); + // check the parallelisms + assertEquals(1, keyExtractor.getParallelism()); + assertEquals(4, reducer.getParallelism()); // check types TypeInformation keyValueInfo = new TupleTypeInfo>>( @@ -232,8 +232,8 @@ public StringValue getKey(Tuple3 value) { @Test public void translateDistinctExpressionKey() { try { - final int DOP = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP); + final int parallelism = 8; + ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); DataSet initialData = getSourcePojoDataSet(env); @@ -253,8 +253,8 @@ public void translateDistinctExpressionKey() { // check keys assertArrayEquals(new int[] {0}, reducer.getKeyColumns(0)); - // DOP was not configured on the operator - assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1); + // parallelism was not configured on the operator + assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1); assertTrue(reducer.getInput() instanceof GenericDataSourceBase); } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java index f9253f826e4e6..b578eb781c57a 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java @@ -45,8 +45,8 @@ public class ReduceTranslationTests implements java.io.Serializable { @Test public void translateNonGroupedReduce() { try { - final int DOP = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP); + final int parallelism = 8; + ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); DataSet> initialData = getSourceDataSet(env); @@ -69,8 +69,8 @@ public Tuple3 reduce(Tuple3); } @@ -84,8 +84,8 @@ public Tuple3 reduce(Tuple3> initialData = getSourceDataSet(env); @@ -108,8 +108,8 @@ public Tuple3 reduce(Tuple3 reduce(Tuple3> initialData = getSourceDataSet(env); @@ -154,10 +154,10 @@ public Tuple3 reduce(Tuple3 reducer = (PlanUnwrappingReduceOperator) keyProjector.getInput(); MapOperatorBase keyExtractor = (MapOperatorBase) reducer.getInput(); - // check the DOPs - assertEquals(1, keyExtractor.getDegreeOfParallelism()); - assertEquals(4, reducer.getDegreeOfParallelism()); - assertEquals(4, keyProjector.getDegreeOfParallelism()); + // check the parallelisms + assertEquals(1, keyExtractor.getParallelism()); + assertEquals(4, reducer.getParallelism()); + assertEquals(4, keyProjector.getParallelism()); // check types TypeInformation keyValueInfo = new TupleTypeInfo>>( diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java index 2101428888783..90421b7dec1a4 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java @@ -288,9 +288,9 @@ public class Optimizer { private final CostEstimator costEstimator; /** - * The default degree of parallelism for jobs compiled by this compiler. + * The default parallelism for jobs compiled by this compiler. */ - private int defaultDegreeOfParallelism; + private int defaultParallelism; // ------------------------------------------------------------------------ @@ -348,14 +348,14 @@ public Optimizer(DataStatistics stats, CostEstimator estimator) { this.costEstimator = estimator; // determine the default parallelism - this.defaultDegreeOfParallelism = GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY, - ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE); + this.defaultParallelism = GlobalConfiguration.getInteger( + ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM); - if (defaultDegreeOfParallelism < 1) { - LOG.warn("Config value " + defaultDegreeOfParallelism + " for option " - + ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE + " is invalid. Ignoring and using a value of 1."); - this.defaultDegreeOfParallelism = 1; + if (defaultParallelism < 1) { + LOG.warn("Config value " + defaultParallelism + " for option " + + ConfigConstants.DEFAULT_PARALLELISM + " is invalid. Ignoring and using a value of 1."); + this.defaultParallelism = 1; } } @@ -363,13 +363,13 @@ public Optimizer(DataStatistics stats, CostEstimator estimator) { // Getters / Setters // ------------------------------------------------------------------------ - public int getDefaultDegreeOfParallelism() { - return defaultDegreeOfParallelism; + public int getDefaultParallelism() { + return defaultParallelism; } - public void setDefaultDegreeOfParallelism(int defaultDegreeOfParallelism) { - if (defaultDegreeOfParallelism > 0) { - this.defaultDegreeOfParallelism = defaultDegreeOfParallelism; + public void setDefaultParallelism(int defaultParallelism) { + if (defaultParallelism > 0) { + this.defaultParallelism = defaultParallelism; } else { throw new IllegalArgumentException("Default parallelism cannot be zero or negative."); } @@ -435,7 +435,7 @@ private OptimizedPlan compile(Plan program, OptimizerPostPass postPasser) throws final ExecutionMode defaultDataExchangeMode = program.getExecutionConfig().getExecutionMode(); final int defaultParallelism = program.getDefaultParallelism() > 0 ? - program.getDefaultParallelism() : this.defaultDegreeOfParallelism; + program.getDefaultParallelism() : this.defaultParallelism; // log the default settings LOG.debug("Using a default parallelism of {}", defaultParallelism); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java index 068799ef2a0f6..1600a50be55f3 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java @@ -122,12 +122,12 @@ public List getAlternativePlans(CostEstimator estimator) { final ExecutionMode input1Mode = this.input1.getDataExchangeMode(); final ExecutionMode input2Mode = this.input2.getDataExchangeMode(); - final int dop = getParallelism(); - final int inDop1 = getFirstPredecessorNode().getParallelism(); - final int inDop2 = getSecondPredecessorNode().getParallelism(); + final int parallelism = getParallelism(); + final int inParallelism1 = getFirstPredecessorNode().getParallelism(); + final int inParallelism2 = getSecondPredecessorNode().getParallelism(); - final boolean dopChange1 = dop != inDop1; - final boolean dopChange2 = dop != inDop2; + final boolean dopChange1 = parallelism != inParallelism1; + final boolean dopChange2 = parallelism != inParallelism2; final boolean input1breakPipeline = this.input1.isBreakingPipeline(); final boolean input2breakPipeline = this.input2.isBreakingPipeline(); @@ -152,8 +152,8 @@ public List getAlternativePlans(CostEstimator estimator) { // free to choose the ship strategy igps.parameterizeChannel(c1, dopChange1, input1Mode, input1breakPipeline); - // if the DOP changed, make sure that we cancel out properties, unless the - // ship strategy preserves/establishes them even under changing DOPs + // if the parallelism changed, make sure that we cancel out properties, unless the + // ship strategy preserves/establishes them even under changing parallelisms if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) { c1.getGlobalProperties().reset(); } @@ -179,8 +179,8 @@ public List getAlternativePlans(CostEstimator estimator) { // free to choose the ship strategy igps.parameterizeChannel(c2, dopChange2, input2Mode, input2breakPipeline); - // if the DOP changed, make sure that we cancel out properties, unless the - // ship strategy preserves/establishes them even under changing DOPs + // if the parallelism changed, make sure that we cancel out properties, unless the + // ship strategy preserves/establishes them even under changing parallelisms if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) { c2.getGlobalProperties().reset(); } diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java index 55b878524bac2..5dd868ee67b43 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java @@ -131,14 +131,14 @@ public OptimizerNode getNextPartialSolution() { */ public void setNextPartialSolution(OptimizerNode nextPartialSolution, OptimizerNode terminationCriterion) { - // check if the root of the step function has the same DOP as the iteration + // check if the root of the step function has the same parallelism as the iteration // or if the step function has any operator at all if (nextPartialSolution.getParallelism() != getParallelism() || nextPartialSolution == partialSolution || nextPartialSolution instanceof BinaryUnionNode) { // add a no-op to the root to express the re-partitioning NoOpNode noop = new NoOpNode(); - noop.setDegreeOfParallelism(getParallelism()); + noop.setParallelism(getParallelism()); DagConnection noOpConn = new DagConnection(nextPartialSolution, noop, ExecutionMode.PIPELINED); noop.setIncomingConnection(noOpConn); @@ -323,7 +323,7 @@ else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) { locPropsReq.parameterizeChannel(toNoOp); UnaryOperatorNode rebuildPropertiesNode = new UnaryOperatorNode("Rebuild Partial Solution Properties", FieldList.EMPTY_LIST); - rebuildPropertiesNode.setDegreeOfParallelism(candidate.getParallelism()); + rebuildPropertiesNode.setParallelism(candidate.getParallelism()); SingleInputPlanNode rebuildPropertiesPlanNode = new SingleInputPlanNode(rebuildPropertiesNode, "Rebuild Partial Solution Properties", toNoOp, DriverStrategy.UNARY_NO_OP); rebuildPropertiesPlanNode.initProperties(toNoOp.getGlobalProperties(), toNoOp.getLocalProperties()); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java index dbe04f4ecdb55..6ca1149b8c5bf 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java @@ -204,11 +204,11 @@ public List getAlternativePlans(CostEstimator estimator) { List subPlans = getPredecessorNode().getAlternativePlans(estimator); List outputPlans = new ArrayList(); - final int dop = getParallelism(); + final int parallelism = getParallelism(); final int inDop = getPredecessorNode().getParallelism(); final ExecutionMode executionMode = this.input.getDataExchangeMode(); - final boolean dopChange = dop != inDop; + final boolean dopChange = parallelism != inDop; final boolean breakPipeline = this.input.isBreakingPipeline(); InterestingProperties ips = this.input.getInterestingProperties(); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java index e4b35b76bc275..6010f6a288d2b 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java @@ -75,7 +75,7 @@ public DataSourceNode(GenericDataSourceBase pactContract) { } if (NonParallelInput.class.isAssignableFrom(pactContract.getUserCodeWrapper().getUserCodeClass())) { - setDegreeOfParallelism(1); + setParallelism(1); this.sequentialInput = true; } else { this.sequentialInput = false; @@ -115,10 +115,10 @@ public String getName() { } @Override - public void setDegreeOfParallelism(int degreeOfParallelism) { + public void setParallelism(int parallelism) { // if unsplittable, parallelism remains at 1 if (!this.sequentialInput) { - super.setDegreeOfParallelism(degreeOfParallelism); + super.setParallelism(parallelism); } } diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java index 564c0d3ed8af1..d25fed9188905 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java @@ -45,7 +45,7 @@ public GroupCombineNode(GroupCombineOperatorBase operator) { if (this.keys == null) { // case of a key-less reducer. force a parallelism of 1 - setDegreeOfParallelism(1); + setParallelism(1); } this.possibleProperties = initPossibleProperties(); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java index 77acae504deb3..227b75fac3596 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java @@ -53,7 +53,7 @@ public GroupReduceNode(GroupReduceOperatorBase operator) { if (this.keys == null) { // case of a key-less reducer. force a parallelism of 1 - setDegreeOfParallelism(1); + setParallelism(1); } this.possibleProperties = initPossibleProperties(operator.getCustomPartitioner()); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java index 0cad34ec5eacd..6bf43ea277bb1 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java @@ -99,7 +99,7 @@ public abstract class OptimizerNode implements Visitable, Estimat // --------------------------------- General Parameters --------------------------------------- private int parallelism = -1; // the number of parallel instances of this node - + private long minimalMemoryPerSubTask = -1; protected int id = -1; // the id for this node. @@ -390,9 +390,9 @@ public int getParallelism() { * @param parallelism The parallelism to set. * @throws IllegalArgumentException If the parallelism is smaller than one and not -1. */ - public void setDegreeOfParallelism(int parallelism) { + public void setParallelism(int parallelism) { if (parallelism < 1 && parallelism != -1) { - throw new IllegalArgumentException("Degree of parallelism of " + parallelism + " is invalid."); + throw new IllegalArgumentException("Parallelism of " + parallelism + " is invalid."); } this.parallelism = parallelism; } diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java index 14770383ff1b9..52bfb6ae98853 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java @@ -43,7 +43,7 @@ public ReduceNode(ReduceOperatorBase operator) { if (this.keys == null) { // case of a key-less reducer. force a parallelism of 1 - setDegreeOfParallelism(1); + setParallelism(1); } OperatorDescriptorSingle props = this.keys == null ? diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java index cc12bb8ce1acd..e9b31f477b419 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java @@ -283,9 +283,10 @@ public List getAlternativePlans(CostEstimator estimator) { final ExecutionMode executionMode = this.inConn.getDataExchangeMode(); - final int dop = getParallelism(); - final int inDop = getPredecessorNode().getParallelism(); - final boolean dopChange = inDop != dop; + final int parallelism = getParallelism(); + final int inParallelism = getPredecessorNode().getParallelism(); + + final boolean parallelismChange = inParallelism != parallelism; final boolean breaksPipeline = this.inConn.isBreakingPipeline(); @@ -293,8 +294,8 @@ public List getAlternativePlans(CostEstimator estimator) { for (PlanNode child : subPlans) { if (child.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if DOP is not changed - if (dopChange) { + // fully replicated input is always locally forwarded if the parallelism is not changed + if (parallelismChange) { // can not continue with this child childrenSkippedDueToReplicatedInput = true; continue; @@ -307,11 +308,11 @@ public List getAlternativePlans(CostEstimator estimator) { // pick the strategy ourselves for (RequestedGlobalProperties igps: intGlobal) { final Channel c = new Channel(child, this.inConn.getMaterializationMode()); - igps.parameterizeChannel(c, dopChange, executionMode, breaksPipeline); + igps.parameterizeChannel(c, parallelismChange, executionMode, breaksPipeline); - // if the DOP changed, make sure that we cancel out properties, unless the - // ship strategy preserves/establishes them even under changing DOPs - if (dopChange && !c.getShipStrategy().isNetworkStrategy()) { + // if the parallelism changed, make sure that we cancel out properties, unless the + // ship strategy preserves/establishes them even under changing parallelisms + if (parallelismChange && !c.getShipStrategy().isNetworkStrategy()) { c.getGlobalProperties().reset(); } @@ -339,7 +340,7 @@ public List getAlternativePlans(CostEstimator estimator) { c.setShipStrategy(shipStrategy, exMode); } - if (dopChange) { + if (parallelismChange) { c.adjustGlobalPropertiesForFullParallelismChange(); } diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java index 40725ba18f217..06606f0dd6d56 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java @@ -47,7 +47,7 @@ public SinkJoiner(OptimizerNode input1, OptimizerNode input2) { this.input1 = conn1; this.input2 = conn2; - setDegreeOfParallelism(1); + setParallelism(1); } @Override diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java index 39da1654a46e8..f3122ba2bf1f5 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java @@ -352,12 +352,12 @@ public List getAlternativePlans(CostEstimator estimator) { final ExecutionMode input1Mode = this.input1.getDataExchangeMode(); final ExecutionMode input2Mode = this.input2.getDataExchangeMode(); - final int dop = getParallelism(); - final int inDop1 = getFirstPredecessorNode().getParallelism(); - final int inDop2 = getSecondPredecessorNode().getParallelism(); + final int parallelism = getParallelism(); + final int inParallelism1 = getFirstPredecessorNode().getParallelism(); + final int inParallelism2 = getSecondPredecessorNode().getParallelism(); - final boolean dopChange1 = dop != inDop1; - final boolean dopChange2 = dop != inDop2; + final boolean dopChange1 = parallelism != inParallelism1; + final boolean dopChange2 = parallelism != inParallelism2; final boolean input1breaksPipeline = this.input1.isBreakingPipeline(); final boolean input2breaksPipeline = this.input2.isBreakingPipeline(); @@ -369,7 +369,7 @@ public List getAlternativePlans(CostEstimator estimator) { for (PlanNode child1 : subPlans1) { if (child1.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if DOP is not changed + // fully replicated input is always locally forwarded if parallelism is not changed if (dopChange1) { // can not continue with this child childrenSkippedDueToReplicatedInput = true; @@ -382,7 +382,7 @@ public List getAlternativePlans(CostEstimator estimator) { for (PlanNode child2 : subPlans2) { if (child2.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if DOP is not changed + // fully replicated input is always locally forwarded if parallelism is not changed if (dopChange2) { // can not continue with this child childrenSkippedDueToReplicatedInput = true; @@ -405,8 +405,8 @@ public List getAlternativePlans(CostEstimator estimator) { // free to choose the ship strategy igps1.parameterizeChannel(c1, dopChange1, input1Mode, input1breaksPipeline); - // if the DOP changed, make sure that we cancel out properties, unless the - // ship strategy preserves/establishes them even under changing DOPs + // if the parallelism changed, make sure that we cancel out properties, unless the + // ship strategy preserves/establishes them even under changing parallelisms if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) { c1.getGlobalProperties().reset(); } @@ -434,8 +434,8 @@ public List getAlternativePlans(CostEstimator estimator) { // free to choose the ship strategy igps2.parameterizeChannel(c2, dopChange2, input2Mode, input2breaksPipeline); - // if the DOP changed, make sure that we cancel out properties, unless the - // ship strategy preserves/establishes them even under changing DOPs + // if the parallelism changed, make sure that we cancel out properties, unless the + // ship strategy preserves/establishes them even under changing parallelisms if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) { c2.getGlobalProperties().reset(); } diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java index e85f289255a9d..99c868c29d61b 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java @@ -167,7 +167,7 @@ public void setNextPartialSolution(OptimizerNode solutionSetDelta, OptimizerNode // if the next workset is equal to the workset, we need to inject a no-op node if (nextWorkset == worksetNode || nextWorkset instanceof BinaryUnionNode) { NoOpNode noop = new NoOpNode(); - noop.setDegreeOfParallelism(getParallelism()); + noop.setParallelism(getParallelism()); DagConnection noOpConn = new DagConnection(nextWorkset, noop, executionMode); noop.setIncomingConnection(noOpConn); @@ -179,7 +179,7 @@ public void setNextPartialSolution(OptimizerNode solutionSetDelta, OptimizerNode // attach an extra node to the solution set delta for the cases where we need to repartition UnaryOperatorNode solutionSetDeltaUpdateAux = new UnaryOperatorNode("Solution-Set Delta", getSolutionSetKeyFields(), new SolutionSetDeltaOperator(getSolutionSetKeyFields())); - solutionSetDeltaUpdateAux.setDegreeOfParallelism(getParallelism()); + solutionSetDeltaUpdateAux.setParallelism(getParallelism()); DagConnection conn = new DagConnection(solutionSetDelta, solutionSetDeltaUpdateAux, executionMode); solutionSetDeltaUpdateAux.setIncomingConnection(conn); @@ -371,7 +371,7 @@ else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) { UnaryOperatorNode rebuildWorksetPropertiesNode = new UnaryOperatorNode("Rebuild Workset Properties", FieldList.EMPTY_LIST); - rebuildWorksetPropertiesNode.setDegreeOfParallelism(candidate.getParallelism()); + rebuildWorksetPropertiesNode.setParallelism(candidate.getParallelism()); SingleInputPlanNode rebuildWorksetPropertiesPlanNode = new SingleInputPlanNode( rebuildWorksetPropertiesNode, "Rebuild Workset Properties", @@ -563,7 +563,7 @@ public static class SingleRootJoiner extends TwoInputNode { SingleRootJoiner() { super(new NoOpBinaryUdfOp(new NothingTypeInfo())); - setDegreeOfParallelism(1); + setParallelism(1); } public void setInputs(DagConnection input1, DagConnection input2) { diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java index 8c3f6bde741d9..3646d74e5b0d2 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java @@ -340,7 +340,7 @@ else if (this.partitioning == PartitioningProperty.CUSTOM_PARTITIONING) { * the desired global properties. * * @param channel The channel to parametrize. - * @param globalDopChange Flag indicating whether the degree of parallelism changes + * @param globalDopChange Flag indicating whether the parallelism changes * between sender and receiver. * @param exchangeMode The mode of data exchange (pipelined, always batch, * batch only on shuffle, ...) diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java index b3c083ae578b4..4990a5d662d7d 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java @@ -53,9 +53,9 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { Channel toCombiner = new Channel(in.getSource()); toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - // create an input node for combine with same DOP as input node + // create an input node for combine with same parallelism as input node GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode(); - combinerNode.setDegreeOfParallelism(in.getSource().getParallelism()); + combinerNode.setParallelism(in.getSource().getParallelism()); SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getOperator().getName()+")", toCombiner, DriverStrategy.ALL_GROUP_REDUCE_COMBINE); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java index a172a602862ba..bd600e4a988eb 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java @@ -52,9 +52,9 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { Channel toCombiner = new Channel(in.getSource()); toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - // create an input node for combine with same DOP as input node + // create an input node for combine with same parallelism as input node ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode(); - combinerNode.setDegreeOfParallelism(in.getSource().getParallelism()); + combinerNode.setParallelism(in.getSource().getParallelism()); SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getOperator().getName()+")", toCombiner, DriverStrategy.ALL_REDUCE); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java index b648386fe3858..64054a2b27480 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java @@ -71,7 +71,7 @@ public DriverStrategy getStrategy() { @Override public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - node.setDegreeOfParallelism(in.getSource().getParallelism()); + node.setParallelism(in.getSource().getParallelism()); // sorting key info SingleInputPlanNode singleInputPlanNode = new SingleInputPlanNode( diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java index c4f47d3a3878b..86863d2b46d20 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java @@ -105,9 +105,9 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { Channel toCombiner = new Channel(in.getSource()); toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - // create an input node for combine with same DOP as input node + // create an input node for combine with same parallelism as input node GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode(); - combinerNode.setDegreeOfParallelism(in.getSource().getParallelism()); + combinerNode.setParallelism(in.getSource().getParallelism()); SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine("+node.getOperator() .getName()+")", toCombiner, DriverStrategy.SORTED_GROUP_COMBINE); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java index 2bde29bba2187..e4e6a7f1d10bc 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java @@ -47,9 +47,9 @@ public DriverStrategy getStrategy() { @Override public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - // create in input node for combine with same DOP as input node + // create in input node for combine with the same parallelism as input node GroupReduceNode combinerNode = new GroupReduceNode((GroupReduceOperatorBase) node.getOperator()); - combinerNode.setDegreeOfParallelism(in.getSource().getParallelism()); + combinerNode.setParallelism(in.getSource().getParallelism()); SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine("+node.getOperator().getName()+")", in, DriverStrategy.SORTED_GROUP_COMBINE); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java index 5bb51f3caf8b8..81afe1e3dbaa8 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java @@ -69,9 +69,9 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { Channel toCombiner = new Channel(in.getSource()); toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - // create an input node for combine with same DOP as input node + // create an input node for combine with same parallelism as input node ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode(); - combinerNode.setDegreeOfParallelism(in.getSource().getParallelism()); + combinerNode.setParallelism(in.getSource().getParallelism()); SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getOperator().getName()+")", toCombiner, diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java index 875d1c306a2a4..4f8b1bee2ea5d 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java @@ -473,7 +473,7 @@ private void computeLocalPropertiesAfterShippingOnly() { public void adjustGlobalPropertiesForFullParallelismChange() { if (this.shipStrategy == null || this.shipStrategy == ShipStrategyType.NONE) { - throw new IllegalStateException("Cannot adjust channel for degree of parallelism " + + throw new IllegalStateException("Cannot adjust channel for parallelism " + "change before the ship strategy is set."); } diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java index 451484d482f44..c93d8c2569734 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java @@ -40,7 +40,7 @@ public SinkJoinerPlanNode(SinkJoiner template, Channel input1, Channel input2) { public void setCosts(Costs nodeCosts) { // the plan enumeration logic works as for regular two-input-operators, which is important // because of the branch handling logic. it does pick redistributing network channels - // between the sink and the sink joiner, because sinks joiner has a different DOP than the sink. + // between the sink and the sink joiner, because sinks joiner has a different parallelism than the sink. // we discard any cost and simply use the sum of the costs from the two children. Costs totalCosts = getInput1().getSource().getCumulativeCosts().clone(); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java index 6f918c056d526..b04cdd805bac3 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java @@ -275,7 +275,7 @@ else if (n instanceof BinaryUnionNode) { // output node contents writer.print(",\n\t\t\"contents\": \"" + contents + "\""); - // degree of parallelism + // parallelism writer.print(",\n\t\t\"parallelism\": \"" + (n.getParallelism() >= 1 ? n.getParallelism() : "default") + "\""); 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 04bc5279e02ec..dc21c13795cb2 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 @@ -264,15 +264,15 @@ else if (node instanceof BulkIterationPlanNode) { // for the bulk iteration, we skip creating anything for now. we create the graph // for the step function in the post visit. - // check that the root of the step function has the same DOP as the iteration. - // because the tail must have the same DOP as the head, we can only merge the last - // operator with the tail, if they have the same DOP. not merging is currently not + // check that the root of the step function has the same parallelism as the iteration. + // because the tail must have the same parallelism as the head, we can only merge the last + // operator with the tail, if they have the same parallelism. not merging is currently not // implemented PlanNode root = iterationNode.getRootOfStepFunction(); if (root.getParallelism() != node.getParallelism()) { throw new CompilerException("Error: The final operator of the step " + - "function has a different degree of parallelism than the iteration operator itself."); + "function has a different parallelism than the iteration operator itself."); } IterationDescriptor descr = new IterationDescriptor(iterationNode, this.iterationIdEnumerator++); @@ -289,12 +289,12 @@ else if (node instanceof WorksetIterationPlanNode) { if (nextWorkSet.getParallelism() != node.getParallelism()) { throw new CompilerException("It is currently not supported that the final operator of the step " + - "function has a different degree of parallelism than the iteration operator itself."); + "function has a different parallelism than the iteration operator itself."); } if (solutionSetDelta.getParallelism() != node.getParallelism()) { throw new CompilerException("It is currently not supported that the final operator of the step " + - "function has a different degree of parallelism than the iteration operator itself."); + "function has a different parallelism than the iteration operator itself."); } IterationDescriptor descr = new IterationDescriptor(iterationNode, this.iterationIdEnumerator++); @@ -362,7 +362,7 @@ else if (node instanceof WorksetPlanNode) { // check if a vertex was created, or if it was chained or skipped if (vertex != null) { - // set degree of parallelism + // set parallelism int pd = node.getParallelism(); vertex.setParallelism(pd); @@ -370,10 +370,10 @@ else if (node instanceof WorksetPlanNode) { // check whether this vertex is part of an iteration step function if (this.currentIteration != null) { - // check that the task has the same DOP as the iteration as such + // check that the task has the same parallelism as the iteration as such PlanNode iterationNode = (PlanNode) this.currentIteration; if (iterationNode.getParallelism() < pd) { - throw new CompilerException("Error: All functions that are part of an iteration must have the same, or a lower, degree-of-parallelism than the iteration operator."); + throw new CompilerException("Error: All functions that are part of an iteration must have the same, or a lower, parallelism than the iteration operator."); } // store the id of the iterations the step functions participate in @@ -725,7 +725,7 @@ else if (numSenders == 1) { return 1; } else { - throw new CompilerException("Error: A changing degree of parallelism is currently " + + throw new CompilerException("Error: A changing parallelism is currently " + "not supported between tasks within an iteration."); } } else { @@ -880,7 +880,7 @@ private AbstractJobVertex createBulkIterationHead(BulkPartialSolutionPlanNode ps // the step function, if // 1) There is one parent that the partial solution connects to via a forward pattern and no // local strategy - // 2) DOP and the number of subtasks per instance does not change + // 2) parallelism and the number of subtasks per instance does not change // 3) That successor is not a union // 4) That successor is not itself the last node of the step function // 5) There is no local strategy on the edge for the initial partial solution, as @@ -948,7 +948,7 @@ private AbstractJobVertex createWorksetIterationHead(WorksetPlanNode wspn) { // the step function, if // 1) There is one parent that the partial solution connects to via a forward pattern and no // local strategy - // 2) DOP and the number of subtasks per instance does not change + // 2) parallelism and the number of subtasks per instance does not change // 3) That successor is not a union // 4) That successor is not itself the last node of the step function // 5) There is no local strategy on the edge for the initial workset, as diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java index 160ef95aea04e..37cffceb6766e 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java @@ -83,20 +83,20 @@ public class GraphCreatingVisitor implements Visitor> { private final List sinks; // all data sink nodes in the optimizer plan - private final int defaultParallelism; // the default degree of parallelism + private final int defaultParallelism; // the default parallelism private final GraphCreatingVisitor parent; // reference to enclosing creator, in case of a recursive translation private final ExecutionMode defaultDataExchangeMode; - private final boolean forceDOP; + private final boolean forceParallelism; public GraphCreatingVisitor(int defaultParallelism, ExecutionMode defaultDataExchangeMode) { this(null, false, defaultParallelism, defaultDataExchangeMode, null); } - private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceDOP, int defaultParallelism, + private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceParallelism, int defaultParallelism, ExecutionMode dataExchangeMode, HashMap, OptimizerNode> closure) { if (closure == null){ con2node = new HashMap, OptimizerNode>(); @@ -108,7 +108,7 @@ private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceDOP, int this.defaultParallelism = defaultParallelism; this.parent = parent; this.defaultDataExchangeMode = dataExchangeMode; - this.forceDOP = forceDOP; + this.forceParallelism = forceParallelism; } public List getSinks() { @@ -194,7 +194,7 @@ else if (c instanceof BulkIterationBase.PartialSolutionPlaceHolder) { // catch this for the recursive translation of step functions BulkPartialSolutionNode p = new BulkPartialSolutionNode(holder, containingIterationNode); - p.setDegreeOfParallelism(containingIterationNode.getParallelism()); + p.setParallelism(containingIterationNode.getParallelism()); n = p; } else if (c instanceof DeltaIterationBase.WorksetPlaceHolder) { @@ -209,7 +209,7 @@ else if (c instanceof DeltaIterationBase.WorksetPlaceHolder) { // catch this for the recursive translation of step functions WorksetNode p = new WorksetNode(holder, containingIterationNode); - p.setDegreeOfParallelism(containingIterationNode.getParallelism()); + p.setParallelism(containingIterationNode.getParallelism()); n = p; } else if (c instanceof DeltaIterationBase.SolutionSetPlaceHolder) { @@ -224,7 +224,7 @@ else if (c instanceof DeltaIterationBase.SolutionSetPlaceHolder) { // catch this for the recursive translation of step functions SolutionSetNode p = new SolutionSetNode(holder, containingIterationNode); - p.setDegreeOfParallelism(containingIterationNode.getParallelism()); + p.setParallelism(containingIterationNode.getParallelism()); n = p; } else { @@ -233,13 +233,13 @@ else if (c instanceof DeltaIterationBase.SolutionSetPlaceHolder) { this.con2node.put(c, n); - // set the parallelism only if it has not been set before. some nodes have a fixed DOP, such as the + // set the parallelism only if it has not been set before. some nodes have a fixed parallelism, such as the // key-less reducer (all-reduce) if (n.getParallelism() < 1) { - // set the degree of parallelism - int par = c.getDegreeOfParallelism(); + // set the parallelism + int par = c.getParallelism(); if (par > 0) { - if (this.forceDOP && par != this.defaultParallelism) { + if (this.forceParallelism && par != this.defaultParallelism) { par = this.defaultParallelism; Optimizer.LOG.warn("The parallelism of nested dataflows (such as step functions in iterations) is " + "currently fixed to the parallelism of the surrounding operator (the iteration)."); @@ -247,7 +247,7 @@ else if (c instanceof DeltaIterationBase.SolutionSetPlaceHolder) { } else { par = this.defaultParallelism; } - n.setDegreeOfParallelism(par); + n.setParallelism(par); } return true; diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java index 3e7da6c8f62e1..47efeb1523adf 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java @@ -203,7 +203,7 @@ else if(s.getName().equals("smallFile")) { private Plan getTestPlanRightStatic(String strategy) { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DEFAULT_PARALLELISM); + env.setParallelism(DEFAULT_PARALLELISM); DataSet> bigInput = env.readCsvFile("file://bigFile").types(Long.class, Long.class, Long.class).name("bigFile"); @@ -231,7 +231,7 @@ private Plan getTestPlanRightStatic(String strategy) { private Plan getTestPlanLeftStatic(String strategy) { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DEFAULT_PARALLELISM); + env.setParallelism(DEFAULT_PARALLELISM); @SuppressWarnings("unchecked") DataSet> bigInput = env.fromElements(new Tuple3(1L, 2L, 3L), diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java index 565d9924ac268..4eed23673b335 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java @@ -71,10 +71,10 @@ public abstract class CompilerTestBase implements java.io.Serializable { public void setup() { this.dataStats = new DataStatistics(); this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator()); - this.withStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM); + this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator()); - this.noStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM); + this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); } // ------------------------------------------------------------------------ diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java index 34aa9f842f25f..3b7eae7b3a3fd 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java @@ -42,7 +42,7 @@ public class DistinctCompilationTest extends CompilerTestBase implements java.io public void testDistinctPlain() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); DataSet> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class) .name("source").setParallelism(6); @@ -77,7 +77,7 @@ public void testDistinctPlain() { assertEquals(new FieldList(0, 1), combineNode.getKeys(0)); assertEquals(new FieldList(0, 1), reduceNode.getInput().getLocalStrategyKeys()); - // check DOP + // check parallelism assertEquals(6, sourceNode.getParallelism()); assertEquals(6, combineNode.getParallelism()); assertEquals(8, reduceNode.getParallelism()); @@ -94,7 +94,7 @@ public void testDistinctPlain() { public void testDistinctWithSelectorFunctionKey() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); DataSet> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class) .name("source").setParallelism(6); @@ -135,7 +135,7 @@ public void testDistinctWithSelectorFunctionKey() { assertEquals(new FieldList(0), combineNode.getKeys(0)); assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys()); - // check DOP + // check parallelism assertEquals(6, sourceNode.getParallelism()); assertEquals(6, keyExtractor.getParallelism()); assertEquals(6, combineNode.getParallelism()); @@ -155,7 +155,7 @@ public void testDistinctWithSelectorFunctionKey() { public void testDistinctWithFieldPositionKeyCombinable() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); DataSet> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class) .name("source").setParallelism(6); @@ -191,7 +191,7 @@ public void testDistinctWithFieldPositionKeyCombinable() { assertEquals(new FieldList(1), combineNode.getKeys(0)); assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys()); - // check DOP + // check parallelism assertEquals(6, sourceNode.getParallelism()); assertEquals(6, combineNode.getParallelism()); assertEquals(8, reduceNode.getParallelism()); diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java index ac4f820b6b846..810ec0e3a69bc 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java @@ -91,7 +91,7 @@ public void testSolutionSetDeltaDependsOnBroadcastVariable() { public void testTwoIterationsWithMapperInbetween() throws Exception { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); DataSet> verticesWithInitialId = env.fromElements(new Tuple2(1L, 2L)); @@ -129,7 +129,7 @@ public void testTwoIterationsWithMapperInbetween() throws Exception { public void testTwoIterationsDirectlyChained() throws Exception { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); DataSet> verticesWithInitialId = env.fromElements(new Tuple2(1L, 2L)); @@ -165,7 +165,7 @@ public void testTwoIterationsDirectlyChained() throws Exception { public void testTwoWorksetIterationsDirectlyChained() throws Exception { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); DataSet> verticesWithInitialId = env.fromElements(new Tuple2(1L, 2L)); @@ -201,7 +201,7 @@ public void testTwoWorksetIterationsDirectlyChained() throws Exception { public void testIterationPushingWorkOut() throws Exception { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); DataSet> input1 = env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue()); @@ -235,7 +235,7 @@ public void testIterationPushingWorkOut() throws Exception { public void testWorksetIterationPipelineBreakerPlacement() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(8); + env.setParallelism(8); // the workset (input two of the delta iteration) is the same as what is consumed be the successive join DataSet> initialWorkset = env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue()); diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DOPChangeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java similarity index 85% rename from flink-optimizer/src/test/java/org/apache/flink/optimizer/DOPChangeTest.java rename to flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java index b17e7770000bc..a54136a39ac5f 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DOPChangeTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java @@ -17,13 +17,6 @@ */ package org.apache.flink.optimizer; -import org.junit.Assert; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.record.operators.FileDataSink; -import org.apache.flink.api.java.record.operators.FileDataSource; -import org.apache.flink.api.java.record.operators.JoinOperator; -import org.apache.flink.api.java.record.operators.MapOperator; -import org.apache.flink.api.java.record.operators.ReduceOperator; import org.apache.flink.optimizer.plan.Channel; import org.apache.flink.optimizer.plan.DualInputPlanNode; import org.apache.flink.optimizer.plan.OptimizedPlan; @@ -36,6 +29,13 @@ import org.apache.flink.optimizer.util.DummyOutputFormat; import org.apache.flink.optimizer.util.IdentityMap; import org.apache.flink.optimizer.util.IdentityReduce; +import org.junit.Assert; +import org.apache.flink.api.common.Plan; +import org.apache.flink.api.java.record.operators.FileDataSink; +import org.apache.flink.api.java.record.operators.FileDataSource; +import org.apache.flink.api.java.record.operators.JoinOperator; +import org.apache.flink.api.java.record.operators.MapOperator; +import org.apache.flink.api.java.record.operators.ReduceOperator; import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.types.IntValue; @@ -45,17 +45,17 @@ /** * Tests in this class: *

    - *
  • Tests that check the correct handling of the properties and strategies in the case where the degree of + *
  • Tests that check the correct handling of the properties and strategies in the case where the * parallelism between tasks is increased or decreased. *
*/ @SuppressWarnings({"serial", "deprecation"}) -public class DOPChangeTest extends CompilerTestBase { +public class ParallelismChangeTest extends CompilerTestBase { /** * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties). * - * Increases DOP between 1st reduce and 2nd map, so the hash partitioning from 1st reduce is not reusable. + * Increases parallelism between 1st reduce and 2nd map, so the hash partitioning from 1st reduce is not reusable. * Expected to re-establish partitioning between reduce and map, via hash, because random is a full network * transit as well. */ @@ -65,29 +65,29 @@ public void checkPropertyHandlingWithIncreasingGlobalParallelism1() { // construct the plan FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source"); - source.setDegreeOfParallelism(degOfPar); + source.setParallelism(degOfPar); MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build(); - map1.setDegreeOfParallelism(degOfPar); + map1.setParallelism(degOfPar); map1.setInput(source); ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build(); - reduce1.setDegreeOfParallelism(degOfPar); + reduce1.setParallelism(degOfPar); reduce1.setInput(map1); MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build(); - map2.setDegreeOfParallelism(degOfPar * 2); + map2.setParallelism(degOfPar * 2); map2.setInput(reduce1); ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build(); - reduce2.setDegreeOfParallelism(degOfPar * 2); + reduce2.setParallelism(degOfPar * 2); reduce2.setInput(map2); FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink"); - sink.setDegreeOfParallelism(degOfPar * 2); + sink.setParallelism(degOfPar * 2); sink.setInput(reduce2); - Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism"); + Plan plan = new Plan(sink, "Test Increasing parallelism"); // submit the plan to the compiler OptimizedPlan oPlan = compileNoStats(plan); @@ -110,7 +110,7 @@ public void checkPropertyHandlingWithIncreasingGlobalParallelism1() { /** * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties). * - * Increases DOP between 2nd map and 2nd reduce, so the hash partitioning from 1st reduce is not reusable. + * Increases parallelism between 2nd map and 2nd reduce, so the hash partitioning from 1st reduce is not reusable. * Expected to re-establish partitioning between map and reduce (hash). */ @Test @@ -119,29 +119,29 @@ public void checkPropertyHandlingWithIncreasingGlobalParallelism2() { // construct the plan FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source"); - source.setDegreeOfParallelism(degOfPar); + source.setParallelism(degOfPar); MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build(); - map1.setDegreeOfParallelism(degOfPar); + map1.setParallelism(degOfPar); map1.setInput(source); ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build(); - reduce1.setDegreeOfParallelism(degOfPar); + reduce1.setParallelism(degOfPar); reduce1.setInput(map1); MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build(); - map2.setDegreeOfParallelism(degOfPar); + map2.setParallelism(degOfPar); map2.setInput(reduce1); ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build(); - reduce2.setDegreeOfParallelism(degOfPar * 2); + reduce2.setParallelism(degOfPar * 2); reduce2.setInput(map2); FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink"); - sink.setDegreeOfParallelism(degOfPar * 2); + sink.setParallelism(degOfPar * 2); sink.setInput(reduce2); - Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism"); + Plan plan = new Plan(sink, "Test Increasing parallelism"); // submit the plan to the compiler OptimizedPlan oPlan = compileNoStats(plan); @@ -164,7 +164,7 @@ public void checkPropertyHandlingWithIncreasingGlobalParallelism2() { /** * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties). * - * Increases DOP between 1st reduce and 2nd map, such that more tasks are on one instance. + * Increases parallelism between 1st reduce and 2nd map, such that more tasks are on one instance. * Expected to re-establish partitioning between map and reduce via a local hash. */ @Test @@ -173,29 +173,29 @@ public void checkPropertyHandlingWithIncreasingLocalParallelism() { // construct the plan FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source"); - source.setDegreeOfParallelism(degOfPar); + source.setParallelism(degOfPar); MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build(); - map1.setDegreeOfParallelism(degOfPar); + map1.setParallelism(degOfPar); map1.setInput(source); ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build(); - reduce1.setDegreeOfParallelism(degOfPar); + reduce1.setParallelism(degOfPar); reduce1.setInput(map1); MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build(); - map2.setDegreeOfParallelism(degOfPar * 2); + map2.setParallelism(degOfPar * 2); map2.setInput(reduce1); ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build(); - reduce2.setDegreeOfParallelism(degOfPar * 2); + reduce2.setParallelism(degOfPar * 2); reduce2.setInput(map2); FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink"); - sink.setDegreeOfParallelism(degOfPar * 2); + sink.setParallelism(degOfPar * 2); sink.setInput(reduce2); - Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism"); + Plan plan = new Plan(sink, "Test Increasing parallelism"); // submit the plan to the compiler OptimizedPlan oPlan = compileNoStats(plan); @@ -219,34 +219,34 @@ public void checkPropertyHandlingWithIncreasingLocalParallelism() { @Test - public void checkPropertyHandlingWithDecreasingDegreeOfParallelism() { + public void checkPropertyHandlingWithDecreasingParallelism() { final int degOfPar = DEFAULT_PARALLELISM; // construct the plan FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source"); - source.setDegreeOfParallelism(degOfPar * 2); + source.setParallelism(degOfPar * 2); MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build(); - map1.setDegreeOfParallelism(degOfPar * 2); + map1.setParallelism(degOfPar * 2); map1.setInput(source); ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build(); - reduce1.setDegreeOfParallelism(degOfPar * 2); + reduce1.setParallelism(degOfPar * 2); reduce1.setInput(map1); MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build(); - map2.setDegreeOfParallelism(degOfPar); + map2.setParallelism(degOfPar); map2.setInput(reduce1); ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build(); - reduce2.setDegreeOfParallelism(degOfPar); + reduce2.setParallelism(degOfPar); reduce2.setInput(map2); FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink"); - sink.setDegreeOfParallelism(degOfPar); + sink.setParallelism(degOfPar); sink.setInput(reduce2); - Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism"); + Plan plan = new Plan(sink, "Test Increasing parallelism"); // submit the plan to the compiler OptimizedPlan oPlan = compileNoStats(plan); @@ -269,7 +269,7 @@ public void checkPropertyHandlingWithDecreasingDegreeOfParallelism() { } /** - * Checks that re-partitioning happens when the inputs of a two-input contract have different DOPs. + * Checks that re-partitioning happens when the inputs of a two-input contract have different parallelisms. * * Test Plan: *
@@ -302,14 +302,14 @@ public void checkPropertyHandlingWithTwoInputs() {
 		
 		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, mat);
 		
-		sourceA.setDegreeOfParallelism(5);
-		sourceB.setDegreeOfParallelism(7);
-		redA.setDegreeOfParallelism(5);
-		redB.setDegreeOfParallelism(7);
+		sourceA.setParallelism(5);
+		sourceB.setParallelism(7);
+		redA.setParallelism(5);
+		redB.setParallelism(7);
 		
-		mat.setDegreeOfParallelism(5);
+		mat.setParallelism(5);
 		
-		sink.setDegreeOfParallelism(5);
+		sink.setParallelism(5);
 		
 		
 		// return the PACT plan
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java
index 86f01b0605eb7..31f71d168a751 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java
@@ -41,7 +41,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 	public void testPipelineBreakerWithBroadcastVariable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(64);
+			env.setParallelism(64);
 			
 			DataSet source = env.generateSequence(1, 10).map(new IdentityMapper());
 			
@@ -69,7 +69,7 @@ public void testPipelineBreakerWithBroadcastVariable() {
 	public void testPipelineBreakerBroadcastedAllReduce() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(64);
+			env.setParallelism(64);
 			
 			DataSet sourceWithMapper = env.generateSequence(1, 10).map(new IdentityMapper());
 			
@@ -103,7 +103,7 @@ public void testPipelineBreakerBroadcastedAllReduce() {
 	public void testPipelineBreakerBroadcastedPartialSolution() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(64);
+			env.setParallelism(64);
 			
 			
 			DataSet initialSource = env.generateSequence(1, 10);
@@ -144,7 +144,7 @@ public void testPilelineBreakerWithCross() {
 		try {
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet initialSource = env.generateSequence(1, 10);
 				
@@ -166,7 +166,7 @@ public void testPilelineBreakerWithCross() {
 			
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet initialSource = env.generateSequence(1, 10);
 				
@@ -189,7 +189,7 @@ public void testPilelineBreakerWithCross() {
 			
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet initialSource = env.generateSequence(1, 10);
 				
@@ -212,7 +212,7 @@ public void testPilelineBreakerWithCross() {
 			
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet initialSource = env.generateSequence(1, 10);
 				
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java
index 7be2b163d5630..3cf081f4e3f21 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java
@@ -55,7 +55,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -88,7 +88,7 @@ public void checkSinglePartitionedSource1() {
 	public void checkSinglePartitionedSource2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -121,7 +121,7 @@ public void checkSinglePartitionedSource2() {
 	public void checkSinglePartitionedSource3() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -153,7 +153,7 @@ public void checkSinglePartitionedSource3() {
 	public void checkSinglePartitionedSource4() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -185,7 +185,7 @@ public void checkSinglePartitionedSource4() {
 	public void checkSinglePartitionedSource5() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -217,7 +217,7 @@ public void checkSinglePartitionedSource5() {
 	public void checkSinglePartitionedSource6() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -249,7 +249,7 @@ public void checkSinglePartitionedSource6() {
 	public void checkSinglePartitionedSource7() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -283,7 +283,7 @@ public void checkSinglePartitionedSource7() {
 	public void checkSinglePartitionedGroupedSource1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -317,7 +317,7 @@ public void checkSinglePartitionedGroupedSource1() {
 	public void checkSinglePartitionedGroupedSource2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -352,7 +352,7 @@ public void checkSinglePartitionedGroupedSource2() {
 	public void checkSinglePartitionedGroupedSource3() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -386,7 +386,7 @@ public void checkSinglePartitionedGroupedSource3() {
 	public void checkSinglePartitionedGroupedSource4() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -420,7 +420,7 @@ public void checkSinglePartitionedGroupedSource4() {
 	public void checkSinglePartitionedGroupedSource5() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -454,7 +454,7 @@ public void checkSinglePartitionedGroupedSource5() {
 	public void checkSinglePartitionedGroupedSource6() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -488,7 +488,7 @@ public void checkSinglePartitionedGroupedSource6() {
 	public void checkSinglePartitionedGroupedSource7() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -521,7 +521,7 @@ public void checkSinglePartitionedGroupedSource7() {
 	public void checkSinglePartitionedGroupedSource8() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -555,7 +555,7 @@ public void checkSinglePartitionedGroupedSource8() {
 	public void checkSinglePartitionedOrderedSource1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -589,7 +589,7 @@ public void checkSinglePartitionedOrderedSource1() {
 	public void checkSinglePartitionedOrderedSource2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -624,7 +624,7 @@ public void checkSinglePartitionedOrderedSource2() {
 	public void checkSinglePartitionedOrderedSource3() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -658,7 +658,7 @@ public void checkSinglePartitionedOrderedSource3() {
 	public void checkSinglePartitionedOrderedSource4() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -692,7 +692,7 @@ public void checkSinglePartitionedOrderedSource4() {
 	public void checkSinglePartitionedOrderedSource5() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -726,7 +726,7 @@ public void checkSinglePartitionedOrderedSource5() {
 	public void checkSinglePartitionedOrderedSource6() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -759,7 +759,7 @@ public void checkSinglePartitionedOrderedSource6() {
 	public void checkSinglePartitionedOrderedSource7() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -793,7 +793,7 @@ public void checkSinglePartitionedOrderedSource7() {
 	public void checkCoPartitionedSources1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data1 =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -841,7 +841,7 @@ public void checkCoPartitionedSources1() {
 	public void checkCoPartitionedSources2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource> data1 =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
index da44b59559b47..fd451f7e5edbb 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
@@ -53,7 +53,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkJoinWithReplicatedSourceInput() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -89,7 +89,7 @@ public void checkJoinWithReplicatedSourceInput() {
 	public void checkJoinWithReplicatedSourceInputBehindMap() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -126,7 +126,7 @@ public void checkJoinWithReplicatedSourceInputBehindMap() {
 	public void checkJoinWithReplicatedSourceInputBehindFilter() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -163,7 +163,7 @@ public void checkJoinWithReplicatedSourceInputBehindFilter() {
 	public void checkJoinWithReplicatedSourceInputBehindFlatMap() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -200,7 +200,7 @@ public void checkJoinWithReplicatedSourceInputBehindFlatMap() {
 	public void checkJoinWithReplicatedSourceInputBehindMapPartition() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -237,7 +237,7 @@ public void checkJoinWithReplicatedSourceInputBehindMapPartition() {
 	public void checkJoinWithReplicatedSourceInputBehindMultiMaps() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -277,7 +277,7 @@ public void checkJoinWithReplicatedSourceInputBehindMultiMaps() {
 	public void checkCrossWithReplicatedSourceInput() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -313,7 +313,7 @@ public void checkCrossWithReplicatedSourceInput() {
 	public void checkCrossWithReplicatedSourceInputBehindMap() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -345,13 +345,13 @@ public void checkCrossWithReplicatedSourceInputBehindMap() {
 	}
 
 	/**
-	 * Tests compiler fail for join program with replicated data source and changing DOP.
+	 * Tests compiler fail for join program with replicated data source and changing parallelism.
 	 */
 	@Test(expected = CompilerException.class)
-	public void checkJoinWithReplicatedSourceInputChangingDOP() {
+	public void checkJoinWithReplicatedSourceInputChangingparallelism() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -370,13 +370,13 @@ public void checkJoinWithReplicatedSourceInputChangingDOP() {
 	}
 
 	/**
-	 * Tests compiler fail for join program with replicated data source behind map and changing DOP.
+	 * Tests compiler fail for join program with replicated data source behind map and changing parallelism.
 	 */
 	@Test(expected = CompilerException.class)
-	public void checkJoinWithReplicatedSourceInputBehindMapChangingDOP() {
+	public void checkJoinWithReplicatedSourceInputBehindMapChangingparallelism() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -402,7 +402,7 @@ public void checkJoinWithReplicatedSourceInputBehindMapChangingDOP() {
 	@Test(expected = CompilerException.class)
 	public void checkJoinWithReplicatedSourceInputBehindReduce() {
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
@@ -427,7 +427,7 @@ public void checkJoinWithReplicatedSourceInputBehindReduce() {
 	@Test(expected = CompilerException.class)
 	public void checkJoinWithReplicatedSourceInputBehindRebalance() {
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat, FileInputSplit> rif =
 				new ReplicatingInputFormat, FileInputSplit>(new CsvInputFormat>(new Path("/some/path"), String.class));
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java
index d397ea22e8626..f865a9f4656b5 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java
@@ -45,7 +45,7 @@ public void testPartitionTuples() {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet> data = env.fromElements(new Tuple2(0, 0))
 					.rebalance();
@@ -88,7 +88,7 @@ public void testPartitionTuplesInvalidType() {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet> data = env.fromElements(new Tuple2(0, 0))
 					.rebalance();
@@ -115,7 +115,7 @@ public void testPartitionPojo() {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet data = env.fromElements(new Pojo())
 					.rebalance();
@@ -158,7 +158,7 @@ public void testPartitionPojoInvalidType() {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet data = env.fromElements(new Pojo())
 					.rebalance();
@@ -185,7 +185,7 @@ public void testPartitionKeySelector() {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet data = env.fromElements(new Pojo())
 					.rebalance();
@@ -237,7 +237,7 @@ public void testPartitionKeySelectorInvalidType() {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet data = env.fromElements(new Pojo())
 					.rebalance();
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java
index de0283671e9d7..2f9b32fd80591 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java
@@ -39,7 +39,7 @@ public class DistinctAndGroupingOptimizerTest extends CompilerTestBase {
 	public void testDistinctPreservesPartitioningOfDistinctFields() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(4);
+			env.setParallelism(4);
 			
 			@SuppressWarnings("unchecked")
 			DataSet> data = env.fromElements(new Tuple2(0L, 0L), new Tuple2(1L, 1L))
@@ -75,7 +75,7 @@ public void testDistinctPreservesPartitioningOfDistinctFields() {
 	public void testDistinctDestroysPartitioningOfNonDistinctFields() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(4);
+			env.setParallelism(4);
 			
 			@SuppressWarnings("unchecked")
 			DataSet> data = env.fromElements(new Tuple2(0L, 0L), new Tuple2(1L, 1L))
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java
index a683968309ace..c0e2fa711e561 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java
@@ -44,7 +44,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 	public void testAllGroupReduceNoCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source");
 			
@@ -59,7 +59,7 @@ public void reduce(Iterable values, Collector out) {}
 			OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op);
 			
 			
-			// the all-reduce has no combiner, when the DOP of the input is one
+			// the all-reduce has no combiner, when the parallelism of the input is one
 			
 			SourcePlanNode sourceNode = resolver.getNode("source");
 			SingleInputPlanNode reduceNode = resolver.getNode("reducer");
@@ -72,7 +72,7 @@ public void reduce(Iterable values, Collector out) {}
 			// check that reduce has the right strategy
 			assertEquals(DriverStrategy.ALL_GROUP_REDUCE, reduceNode.getDriverStrategy());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(1, sourceNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
 			assertEquals(1, sinkNode.getParallelism());
@@ -88,7 +88,7 @@ public void reduce(Iterable values, Collector out) {}
 	public void testAllReduceWithCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet data = env.generateSequence(1, 8000000).name("source");
 			
@@ -120,7 +120,7 @@ public void reduce(Iterable values, Collector out) {}
 			assertEquals(DriverStrategy.ALL_GROUP_REDUCE, reduceNode.getDriverStrategy());
 			assertEquals(DriverStrategy.ALL_GROUP_REDUCE_COMBINE, combineNode.getDriverStrategy());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(8, sourceNode.getParallelism());
 			assertEquals(8, combineNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
@@ -138,7 +138,7 @@ public void reduce(Iterable values, Collector out) {}
 	public void testGroupedReduceWithFieldPositionKeyNonCombinable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -171,7 +171,7 @@ public void reduce(Iterable> values, Collector> values, Collector> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -228,7 +228,7 @@ public void reduce(Iterable> values, Collector> values, Collector> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -284,7 +284,7 @@ public void reduce(Iterable> values, Collector> values, Collector> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -350,7 +350,7 @@ public void reduce(Iterable> values, Collector iteration = env.generateSequence(-4, 1000).iterate(100);
 			iteration.closeWith(iteration).print();
@@ -65,7 +65,7 @@ public void testIdentityIteration() {
 	public void testEmptyWorksetIteration() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(43);
+			env.setParallelism(43);
 			
 			DataSet> input = env.generateSequence(1, 20)
 					.map(new MapFunction>() {
@@ -93,7 +93,7 @@ public void testEmptyWorksetIteration() {
 	public void testIterationWithUnionRoot() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(43);
+			env.setParallelism(43);
 			
 			IterativeDataSet iteration = env.generateSequence(-4, 1000).iterate(100);
 			
@@ -132,7 +132,7 @@ public void testIterationWithUnionRoot() {
 	public void testWorksetIterationWithUnionRoot() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(43);
+			env.setParallelism(43);
 			
 			DataSet> input = env.generateSequence(1, 20)
 					.map(new MapFunction>() {
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java
index 0724a9f281ce8..e1b18f9bb125c 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java
@@ -42,7 +42,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 	public void testAllReduceNoCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source");
 			
@@ -61,7 +61,7 @@ public Double reduce(Double value1, Double value2){
 			OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op);
 			
 			
-			// the all-reduce has no combiner, when the DOP of the input is one
+			// the all-reduce has no combiner, when the parallelism of the input is one
 			
 			SourcePlanNode sourceNode = resolver.getNode("source");
 			SingleInputPlanNode reduceNode = resolver.getNode("reducer");
@@ -71,7 +71,7 @@ public Double reduce(Double value1, Double value2){
 			assertEquals(sourceNode, reduceNode.getInput().getSource());
 			assertEquals(reduceNode, sinkNode.getInput().getSource());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(1, sourceNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
 			assertEquals(1, sinkNode.getParallelism());
@@ -87,7 +87,7 @@ public Double reduce(Double value1, Double value2){
 	public void testAllReduceWithCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet data = env.generateSequence(1, 8000000).name("source");
 			
@@ -121,7 +121,7 @@ public Long reduce(Long value1, Long value2){
 			assertEquals(DriverStrategy.ALL_REDUCE, reduceNode.getDriverStrategy());
 			assertEquals(DriverStrategy.ALL_REDUCE, combineNode.getDriverStrategy());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(8, sourceNode.getParallelism());
 			assertEquals(8, combineNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
@@ -138,7 +138,7 @@ public Long reduce(Long value1, Long value2){
 	public void testGroupedReduceWithFieldPositionKey() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -179,7 +179,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2 reduce(Tuple2 value1, Tuple2> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -243,7 +243,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2> solutionSetInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Solution Set");
@@ -245,7 +245,7 @@ public Tuple3 join(Tuple3 first, Tuple3> solutionSetInput = env.fromElements(new Tuple3(1L, 2L, 3L)).name("Solution Set");
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 46ec4456e89b4..c744c56ff7a88 100644
--- 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
@@ -71,7 +71,7 @@
  *         The ExecutionJobVertex is identified inside the graph by the {@link JobVertexID}, which it takes
  *         from the JobGraph's corresponding JobVertex.
  *     
  • The {@link ExecutionVertex} represents one parallel subtask. For each ExecutionJobVertex, there are - * as many ExecutionVertices as the degree of parallelism. The ExecutionVertex is identified by + * as many ExecutionVertices as the parallelism. The ExecutionVertex is identified by * the ExecutionJobVertex and the number of the parallel subtask
  • *
  • The {@link Execution} is one attempt to execute a ExecutionVertex. There may be multiple Executions * for the ExecutionVertex, in case of a failure, or in the case where some data needs to be recomputed diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java index 8816a697231eb..c9481553764ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java @@ -194,22 +194,22 @@ public Class getInvokableClass(ClassLoader cl) { } /** - * Gets the degree of parallelism of the task. + * Gets the parallelism of the task. * - * @return The degree of parallelism of the task. + * @return The parallelism of the task. */ public int getParallelism() { return parallelism; } /** - * Sets the degree of parallelism for the task. + * Sets the parallelism for the task. * - * @param parallelism The degree of parallelism for the task. + * @param parallelism The parallelism for the task. */ public void setParallelism(int parallelism) { if (parallelism < 1) { - throw new IllegalArgumentException("The degree of parallelism must be at least one."); + throw new IllegalArgumentException("The parallelism must be at least one."); } this.parallelism = parallelism; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java index fb32a6ea73e78..47b1b96f4f638 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java @@ -36,7 +36,7 @@ public enum ShipStrategyType { FORWARD(false, false), /** - * Repartitioning the data randomly, typically when the degree of parallelism between two nodes changes. + * Repartitioning the data randomly, typically when the parallelism between two nodes changes. */ PARTITION_RANDOM(true, false), 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 dfe6b50e1a0ef..f0001a927efaf 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 @@ -86,7 +86,7 @@ public void testNotEnoughSubtasks() { new TestLocatableInputSplit(3, "host3") }; - // This should fail with an exception, since the DOP of 2 does not + // This should fail with an exception, since the parallelism of 2 does not // support strictly local assignment onto 3 hosts try { runTests(numHosts, slotsPerHost, parallelism, splits); diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java index 7f468fa08eda7..87d77790dec57 100644 --- a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java +++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java @@ -178,8 +178,8 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase ds.setParallelism(dop) - case op: Operator[_, _] => op.setParallelism(dop) - case di: DeltaIterationResultSet[_, _] => di.getIterationHead.parallelism(dop) + case ds: DataSource[_] => ds.setParallelism(parallelism) + case op: Operator[_, _] => op.setParallelism(parallelism) + case di: DeltaIterationResultSet[_, _] => di.getIterationHead.parallelism(parallelism) case _ => throw new UnsupportedOperationException("Operator " + javaSet.toString + " cannot have " + "parallelism.") @@ -164,7 +164,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { } /** - * Returns the degree of parallelism of this operation. + * Returns the parallelism of this operation. */ def getParallelism: Int = javaSet match { case ds: DataSource[_] => ds.getParallelism diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala index cccea78f13102..4c1e6278c6cfa 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala @@ -78,9 +78,9 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { } /** - * Sets the degree of parallelism (DOP) for operations executed through this environment. - * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with - * x parallel instances. This value can be overridden by specific operations using + * Sets the parallelism (parallelism) for operations executed through this environment. + * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run + * with x parallel instances. This value can be overridden by specific operations using * [[DataSet.setParallelism]]. * @deprecated Please use [[setParallelism]] */ @@ -90,8 +90,10 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { } /** - * Returns the default degree of parallelism for this execution environment. Note that this - * value can be overridden by individual operations using [[DataSet.setParallelism] + * Sets the parallelism (parallelism) for operations executed through this environment. + * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run + * with x parallel instances. This value can be overridden by specific operations using + * [[DataSet.setParallelism]]. */ def setParallelism(parallelism: Int): Unit = { javaEnv.setParallelism(parallelism) @@ -432,7 +434,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { * because the framework may move the elements into the cluster if needed. * * Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. */ def fromCollection[T: ClassTag : TypeInformation]( data: Seq[T]): DataSet[T] = { @@ -453,7 +455,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { * framework might move into the cluster if needed. * * Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. */ def fromCollection[T: ClassTag : TypeInformation] ( data: Iterator[T]): DataSet[T] = { @@ -473,7 +475,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { * same type and must be serializable. * * * Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. */ def fromElements[T: ClassTag : TypeInformation](data: T*): DataSet[T] = { Validate.notNull(data, "Data must not be null.") @@ -610,10 +612,10 @@ object ExecutionEnvironment { * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads). */ def createLocalEnvironment( - degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors()) + parallelism: Int = Runtime.getRuntime.availableProcessors()) : ExecutionEnvironment = { val javaEnv = JavaEnv.createLocalEnvironment() - javaEnv.setDegreeOfParallelism(degreeOfParallelism) + javaEnv.setParallelism(parallelism) new ExecutionEnvironment(javaEnv) } @@ -630,8 +632,8 @@ object ExecutionEnvironment { /** * Creates a remote execution environment. The remote environment sends (parts of) the program to * a cluster for execution. Note that all file paths used in the program must be accessible from - * the cluster. The execution will use the cluster's default degree of parallelism, unless the - * parallelism is set explicitly via [[ExecutionEnvironment.setDegreeOfParallelism()]]. + * the cluster. The execution will use the cluster's default parallelism, unless the + * parallelism is set explicitly via [[ExecutionEnvironment.setParallelism()]]. * * @param host The host name or address of the master (JobManager), * where the program should be executed. @@ -649,12 +651,12 @@ object ExecutionEnvironment { /** * Creates a remote execution environment. The remote environment sends (parts of) the program * to a cluster for execution. Note that all file paths used in the program must be accessible - * from the cluster. The execution will use the specified degree of parallelism. + * from the cluster. The execution will use the specified parallelism. * * @param host The host name or address of the master (JobManager), * where the program should be executed. * @param port The port of the master (JobManager), where the program should be executed. - * @param degreeOfParallelism The degree of parallelism to use during the execution. + * @param parallelism The parallelism to use during the execution. * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the * program uses * user-defined functions, user-defined input formats, or any libraries, @@ -664,10 +666,10 @@ object ExecutionEnvironment { def createRemoteEnvironment( host: String, port: Int, - degreeOfParallelism: Int, + parallelism: Int, jarFiles: String*): ExecutionEnvironment = { val javaEnv = JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*) - javaEnv.setDegreeOfParallelism(degreeOfParallelism) + javaEnv.setParallelism(parallelism) new ExecutionEnvironment(javaEnv) } } diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java index 6eaa9ae542385..c54ee0c481ba9 100644 --- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java +++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java @@ -175,17 +175,17 @@ public String getName() { } /** - * Sets the degree of parallelism for the iteration. + * Sets the parallelism for the iteration. * - * @param parallelism The degree of parallelism. + * @param parallelism The parallelism. */ public void setParallelism(int parallelism) { - Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default)."); + Validate.isTrue(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default)."); this.parallelism = parallelism; } /** - * Gets the iteration's degree of parallelism. + * Gets the iteration's parallelism. * * @return The iterations parallelism, or -1, if not set. */ diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java index 18826b6cb1952..6c195cb21ac74 100644 --- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java +++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java @@ -75,7 +75,7 @@ public void testOutDegreesInvalidEdgeSrcId() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(PARALLELISM); + env.setParallelism(PARALLELISM); Graph graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env); @@ -100,7 +100,7 @@ public void testInDegreesInvalidEdgeTrgId() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(PARALLELISM); + env.setParallelism(PARALLELISM); Graph graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env); @@ -125,7 +125,7 @@ public void testGetDegreesInvalidEdgeTrgId() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(PARALLELISM); + env.setParallelism(PARALLELISM); Graph graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env); @@ -150,7 +150,7 @@ public void testGetDegreesInvalidEdgeSrcId() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(PARALLELISM); + env.setParallelism(PARALLELISM); Graph graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env); @@ -175,7 +175,7 @@ public void testGetDegreesInvalidEdgeSrcTrgId() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(PARALLELISM); + env.setParallelism(PARALLELISM); Graph graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeInvalidSrcTrgData(env), env); diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java index b6650d2a03504..bbb75037ff251 100644 --- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java +++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java @@ -31,7 +31,7 @@ public class HadoopMapredITCase extends JavaProgramTestBase { protected void preSubmit() throws Exception { textPath = createTempFile("text.txt", WordCountData.TEXT); resultPath = getTempDirPath("result"); - this.setDegreeOfParallelism(4); + this.setParallelism(4); } @Override diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java index 7eee62909a631..9b4aeea78eb66 100644 --- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java +++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java @@ -32,7 +32,7 @@ public class HadoopInputOutputITCase extends JavaProgramTestBase { protected void preSubmit() throws Exception { textPath = createTempFile("text.txt", WordCountData.TEXT); resultPath = getTempDirPath("result"); - this.setDegreeOfParallelism(4); + this.setParallelism(4); } @Override diff --git a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala index 28fdfa62c1fe5..95e2a257e235a 100644 --- a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala +++ b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala @@ -33,7 +33,7 @@ class PolynomialBaseITCase extends ShouldMatchers { def testMapElementToPolynomialVectorSpace (): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism (2) + env.setParallelism (2) val input = Seq ( LabeledVector (DenseVector (1), 1.0), @@ -64,7 +64,7 @@ class PolynomialBaseITCase extends ShouldMatchers { def testMapVectorToPolynomialVectorSpace(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(2) + env.setParallelism(2) val input = Seq( LabeledVector(DenseVector(2, 3), 1.0), @@ -96,7 +96,7 @@ class PolynomialBaseITCase extends ShouldMatchers { def testReturnEmptyVectorIfDegreeIsZero(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(2) + env.setParallelism(2) val input = Seq( LabeledVector(DenseVector(2, 3), 1.0), diff --git a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala index 6e324cb8159b9..d783ecb12a151 100644 --- a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala +++ b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala @@ -34,7 +34,7 @@ class ALSITCase extends ShouldMatchers { val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(2) + env.setParallelism(2) val als = ALS() .setIterations(iterations) diff --git a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala index eb825b90549c4..15292b77a9497 100644 --- a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala +++ b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala @@ -33,7 +33,7 @@ class MultipleLinearRegressionITCase extends ShouldMatchers { def testEstimationOfLinearFunction(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(2) + env.setParallelism(2) val learner = MultipleLinearRegression() @@ -69,7 +69,7 @@ class MultipleLinearRegressionITCase extends ShouldMatchers { def testEstimationOfCubicFunction(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(2) + env.setParallelism(2) val polynomialBase = PolynomialBase() val learner = MultipleLinearRegression() diff --git a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java index 4f84467c2bcd6..d0549753e9ac1 100644 --- a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java +++ b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java @@ -217,17 +217,17 @@ public String getName() { } /** - * Sets the degree of parallelism for the iteration. + * Sets the parallelism for the iteration. * - * @param parallelism The degree of parallelism. + * @param parallelism The parallelism. */ public void setParallelism(int parallelism) { - Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default)."); + Validate.isTrue(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default)."); this.parallelism = parallelism; } /** - * Gets the iteration's degree of parallelism. + * Gets the iteration's parallelism. * * @return The iterations parallelism, or -1, if not set. */ diff --git a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java index 019345f471aa9..38c26f08d5d2a 100644 --- a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java +++ b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java @@ -48,7 +48,7 @@ public class SpargelCompilerTest extends CompilerTestBase { public void testSpargelCompiler() { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DEFAULT_PARALLELISM); + env.setParallelism(DEFAULT_PARALLELISM); // compose test program { DataSet vertexIds = env.generateSequence(1, 2); @@ -116,7 +116,7 @@ public void testSpargelCompilerWithBroadcastVariable() { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DEFAULT_PARALLELISM); + env.setParallelism(DEFAULT_PARALLELISM); // compose test program { DataSet bcVar = env.fromElements(1L); diff --git a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java index b31618c111c0f..cd48dcd0ac594 100644 --- a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java +++ b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java @@ -49,7 +49,7 @@ public void testTranslationPlainEdges() { ; final int NUM_ITERATIONS = 13; - final int ITERATION_DOP = 77; + final int ITERATION_parallelism = 77; ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -75,7 +75,7 @@ public void testTranslationPlainEdges() { vertexIteration.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcUpdate); vertexIteration.setName(ITERATION_NAME); - vertexIteration.setParallelism(ITERATION_DOP); + vertexIteration.setParallelism(ITERATION_parallelism); vertexIteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator()); @@ -93,7 +93,7 @@ public void testTranslationPlainEdges() { // check the basic iteration properties assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations()); assertArrayEquals(new int[] {0}, resultSet.getKeyPositions()); - assertEquals(ITERATION_DOP, iteration.getParallelism()); + assertEquals(ITERATION_parallelism, iteration.getParallelism()); assertEquals(ITERATION_NAME, iteration.getName()); assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName()); @@ -129,7 +129,7 @@ public void testTranslationPlainEdgesWithForkedBroadcastVariable() { ; final int NUM_ITERATIONS = 13; - final int ITERATION_DOP = 77; + final int ITERATION_parallelism = 77; ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -154,7 +154,7 @@ public void testTranslationPlainEdgesWithForkedBroadcastVariable() { vertexIteration.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcVar); vertexIteration.setName(ITERATION_NAME); - vertexIteration.setParallelism(ITERATION_DOP); + vertexIteration.setParallelism(ITERATION_parallelism); vertexIteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator()); @@ -172,7 +172,7 @@ public void testTranslationPlainEdgesWithForkedBroadcastVariable() { // check the basic iteration properties assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations()); assertArrayEquals(new int[] {0}, resultSet.getKeyPositions()); - assertEquals(ITERATION_DOP, iteration.getParallelism()); + assertEquals(ITERATION_parallelism, iteration.getParallelism()); assertEquals(ITERATION_NAME, iteration.getName()); assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName()); diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java index dd1221d5e5272..a00f501ef1dae 100644 --- a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java +++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java @@ -34,7 +34,7 @@ public static void main(String[] args) throws Exception { return; } - StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setDegreeOfParallelism(4); + StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4); DataStream kafkaStream = env .addSource(new KafkaSource(host + ":" + port, topic, new JavaDefaultStringSchema())); diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java index 1fe759a7a5472..6fae91ac9bba1 100644 --- a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java +++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java @@ -36,7 +36,7 @@ public static void main(String[] args) throws Exception { return; } - StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setDegreeOfParallelism(4); + StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4); @SuppressWarnings({ "unused", "serial" }) DataStream stream1 = env.addSource(new SourceFunction() { diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java index 47c5a33577d53..e4619dbc859e6 100644 --- a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java +++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java @@ -34,7 +34,7 @@ public static void main(String[] args) throws Exception { return; } - StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setDegreeOfParallelism(4); + StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4); DataStream kafkaStream = env .addSource(new PersistentKafkaSource(host + ":" + port, topic, new JavaDefaultStringSchema())); diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java index 134525d130d83..3597c4e649a1b 100644 --- a/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java +++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java @@ -186,7 +186,7 @@ public void cancel() { stream.addSink(new KafkaSink(zookeeperConnectionString, TOPIC, new JavaDefaultStringSchema())); try { - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); } catch (JobExecutionException good) { Throwable t = good.getCause(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java index a6eade86cae01..a9ae77aad91a6 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java @@ -404,7 +404,7 @@ protected CoInvokable getReduceInvokable( dataStream1.streamGraph.addCoTask(returnStream.getId(), functionInvokable, getInputType1(), getInputType2(), outTypeInfo, functionName, - environment.getDegreeOfParallelism()); + environment.getParallelism()); dataStream1.connectGraph(dataStream1, returnStream.getId(), 1); dataStream1.connectGraph(dataStream2, returnStream.getId(), 2); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index b62a6d83feb98..40123f1c53825 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -101,7 +101,7 @@ public class DataStream { protected final StreamExecutionEnvironment environment; protected final Integer id; protected final String type; - protected int degreeOfParallelism; + protected int parallelism; protected List userDefinedNames; protected StreamPartitioner partitioner; @SuppressWarnings("rawtypes") @@ -131,7 +131,7 @@ public DataStream(StreamExecutionEnvironment environment, String operatorType, this.id = counter; this.type = operatorType; this.environment = environment; - this.degreeOfParallelism = environment.getDegreeOfParallelism(); + this.parallelism = environment.getParallelism(); this.streamGraph = environment.getStreamGraph(); this.userDefinedNames = new ArrayList(); this.partitioner = new DistributePartitioner(true); @@ -150,7 +150,7 @@ public DataStream(DataStream dataStream) { this.environment = dataStream.environment; this.id = dataStream.id; this.type = dataStream.type; - this.degreeOfParallelism = dataStream.degreeOfParallelism; + this.parallelism = dataStream.parallelism; this.userDefinedNames = new ArrayList(dataStream.userDefinedNames); this.partitioner = dataStream.partitioner; this.streamGraph = dataStream.streamGraph; @@ -175,12 +175,12 @@ public Integer getId() { } /** - * Gets the degree of parallelism for this operator. + * Gets the parallelism for this operator. * * @return The parallelism set for this operator. */ public int getParallelism() { - return this.degreeOfParallelism; + return this.parallelism; } /** diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java index 0dda976f38332..b8e0a7d3dc173 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java @@ -41,11 +41,11 @@ public DataStreamSource(StreamExecutionEnvironment environment, String operatorT } @Override - public DataStreamSource setParallelism(int dop) { - if (dop > 1 && !isParallel) { + public DataStreamSource setParallelism(int parallelism) { + if (parallelism > 1 && !isParallel) { throw new IllegalArgumentException("Source: " + this.id + " is not a parallel source"); } else { - return (DataStreamSource) super.setParallelism(dop); + return (DataStreamSource) super.setParallelism(parallelism); } } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java index 7832777f20178..2ac60fe54f10a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java @@ -216,7 +216,7 @@ private DiscretizedStream partition(WindowTransformation transformation) { return out; } else if (transformation == WindowTransformation.REDUCEWINDOW && parallelism != discretizedStream.getExecutionEnvironment() - .getDegreeOfParallelism()) { + .getParallelism()) { DiscretizedStream out = transform(transformation, "Window partitioner", getType(), new WindowPartitioner(parallelism)).setParallelism(parallelism); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java index 16284d440c663..8ffe1fc36c718 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java @@ -62,20 +62,20 @@ protected SingleOutputStreamOperator(DataStream dataStream) { } /** - * Sets the degree of parallelism for this operator. The degree must be 1 or + * Sets the parallelism for this operator. The degree must be 1 or * more. * - * @param dop - * The degree of parallelism for this operator. - * @return The operator with set degree of parallelism. + * @param parallelism + * The parallelism for this operator. + * @return The operator with set parallelism. */ - public SingleOutputStreamOperator setParallelism(int dop) { - if (dop < 1) { + public SingleOutputStreamOperator setParallelism(int parallelism) { + if (parallelism < 1) { throw new IllegalArgumentException("The parallelism of an operator must be at least 1."); } - this.degreeOfParallelism = dop; + this.parallelism = parallelism; - streamGraph.setParallelism(id, degreeOfParallelism); + streamGraph.setParallelism(id, parallelism); return this; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java index efbbda93e1a13..784d20c3ffc5f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java @@ -394,7 +394,7 @@ private DiscretizedStream discretize(WindowTransformation transformation, } /** - * Returns the degree of parallelism for the stream discretizer. The + * Returns the parallelism for the stream discretizer. The * returned parallelism is either 1 for for non-parallel global policies (or * when the input stream is non-parallel), environment parallelism for the * policies that can run in parallel (such as, any ditributed policy, reduce @@ -408,7 +408,7 @@ private int getDiscretizerParallelism(WindowTransformation transformation) { return isLocal || (transformation == WindowTransformation.REDUCEWINDOW && WindowUtils .isParallelPolicy(getTrigger(), getEviction(), dataStream.getParallelism())) - || (discretizerKey != null) ? dataStream.environment.getDegreeOfParallelism() : 1; + || (discretizerKey != null) ? dataStream.environment.getParallelism() : 1; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java index 4824fca0dbf75..e1b1453a66989 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java @@ -29,7 +29,7 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment { */ @Override public void execute() throws Exception { - ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(), getDegreeOfParallelism()); + ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(), getParallelism()); } /** @@ -42,6 +42,6 @@ public void execute() throws Exception { @Override public void execute(String jobName) throws Exception { ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(jobName), - getDegreeOfParallelism()); + getParallelism()); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 2eb05ad47292d..3142bddae1a19 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -120,8 +120,8 @@ private void executeRemotely(JobGraph jobGraph) { @Override public String toString() { - return "Remote Environment (" + this.host + ":" + this.port + " - DOP = " - + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + ")"; + return "Remote Environment (" + this.host + ":" + this.port + " - parallelism = " + + (getParallelism() == -1 ? "default" : getParallelism()) + ")"; } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java index 7d41d2a5d323b..7ae78f172334a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java @@ -32,15 +32,15 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { protected List jars; protected Client client; - protected StreamContextEnvironment(Client client, List jars, int dop) { + protected StreamContextEnvironment(Client client, List jars, int parallelism) { this.client = client; this.jars = jars; - if (dop > 0) { - setDegreeOfParallelism(dop); + if (parallelism > 0) { + setParallelism(parallelism); } else { - setDegreeOfParallelism(GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY, - ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE)); + setParallelism(GlobalConfiguration.getInteger( + ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM)); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index ae4e19bf6d9b2..9b04cae23b429 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -64,7 +64,7 @@ */ public abstract class StreamExecutionEnvironment { - private static int defaultLocalDop = Runtime.getRuntime().availableProcessors(); + private static int defaultLocalParallelism = Runtime.getRuntime().availableProcessors(); private long bufferTimeout = 100; @@ -93,9 +93,9 @@ public ExecutionConfig getConfig() { } /** - * Gets the degree of parallelism with which operation are executed by + * Gets the parallelism with which operation are executed by * default. Operations can individually override this value to use a - * specific degree of parallelism. + * specific parallelism. * * @return The parallelism used by operations, unless they * override that value. @@ -252,12 +252,12 @@ public long getBufferTimeout() { * Sets the default parallelism that will be used for the local execution * environment created by {@link #createLocalEnvironment()}. * - * @param degreeOfParallelism - * The degree of parallelism to use as the default local + * @param parallelism + * The parallelism to use as the default local * parallelism. */ - public static void setDefaultLocalParallelism(int degreeOfParallelism) { - defaultLocalDop = degreeOfParallelism; + public static void setDefaultLocalParallelism(int parallelism) { + defaultLocalParallelism = parallelism; } // -------------------------------------------------------------------------------------------- @@ -617,7 +617,7 @@ private DataStreamSource addSource(SourceFunction function, } boolean isParallel = function instanceof ParallelSourceFunction; - int dop = isParallel ? getDegreeOfParallelism() : 1; + int parallelism = isParallel ? getParallelism() : 1; ClosureCleaner.clean(function, true); StreamInvokable sourceInvokable = new SourceInvokable(function); @@ -626,7 +626,7 @@ private DataStreamSource addSource(SourceFunction function, outTypeInfo, sourceInvokable, isParallel); streamGraph.addSourceVertex(returnStream.getId(), sourceInvokable, null, outTypeInfo, - sourceName, dop); + sourceName, parallelism); return returnStream; } @@ -652,7 +652,7 @@ public static StreamExecutionEnvironment getExecutionEnvironment() { if (env instanceof ContextEnvironment) { ContextEnvironment ctx = (ContextEnvironment) env; currentEnvironment = createContextEnvironment(ctx.getClient(), ctx.getJars(), - ctx.getDegreeOfParallelism()); + ctx.getParallelism()); } else if (env instanceof OptimizerPlanEnvironment | env instanceof PreviewPlanEnvironment) { currentEnvironment = new StreamPlanEnvironment(env); } else { @@ -662,38 +662,38 @@ public static StreamExecutionEnvironment getExecutionEnvironment() { } private static StreamExecutionEnvironment createContextEnvironment(Client client, - List jars, int dop) { - return new StreamContextEnvironment(client, jars, dop); + List jars, int parallelism) { + return new StreamContextEnvironment(client, jars, parallelism); } /** * Creates a {@link LocalStreamEnvironment}. The local execution environment * will run the program in a multi-threaded fashion in the same JVM as the - * environment was created in. The default degree of parallelism of the + * environment was created in. The default parallelism of the * local environment is the number of hardware contexts (CPU cores / * threads), unless it was specified differently by - * {@link #setDegreeOfParallelism(int)}. + * {@link #setParallelism(int)}. * * @return A local execution environment. */ public static LocalStreamEnvironment createLocalEnvironment() { - return createLocalEnvironment(defaultLocalDop); + return createLocalEnvironment(defaultLocalParallelism); } /** * Creates a {@link LocalStreamEnvironment}. The local execution environment * will run the program in a multi-threaded fashion in the same JVM as the - * environment was created in. It will use the degree of parallelism + * environment was created in. It will use the parallelism * specified in the parameter. * - * @param degreeOfParallelism - * The degree of parallelism for the local environment. - * @return A local execution environment with the specified degree of + * @param parallelism + * The parallelism for the local environment. + * @return A local execution environment with the specified * parallelism. */ - public static LocalStreamEnvironment createLocalEnvironment(int degreeOfParallelism) { + public static LocalStreamEnvironment createLocalEnvironment(int parallelism) { currentEnvironment = new LocalStreamEnvironment(); - currentEnvironment.setDegreeOfParallelism(degreeOfParallelism); + currentEnvironment.setParallelism(parallelism); return (LocalStreamEnvironment) currentEnvironment; } @@ -703,7 +703,7 @@ public static LocalStreamEnvironment createLocalEnvironment(int degreeOfParallel * (parts of) the program to a cluster for execution. Note that all file * paths used in the program must be accessible from the cluster. The * execution will use no parallelism, unless the parallelism is set - * explicitly via {@link #setDegreeOfParallelism}. + * explicitly via {@link #setParallelism}. * * @param host * The host name or address of the master (JobManager), where the @@ -728,7 +728,7 @@ public static StreamExecutionEnvironment createRemoteEnvironment(String host, in * Creates a {@link RemoteStreamEnvironment}. The remote environment sends * (parts of) the program to a cluster for execution. Note that all file * paths used in the program must be accessible from the cluster. The - * execution will use the specified degree of parallelism. + * execution will use the specified parallelism. * * @param host * The host name or address of the master (JobManager), where the @@ -736,8 +736,8 @@ public static StreamExecutionEnvironment createRemoteEnvironment(String host, in * @param port * The port of the master (JobManager), where the program should * be executed. - * @param degreeOfParallelism - * The degree of parallelism to use during the execution. + * @param parallelism + * The parallelism to use during the execution. * @param jarFiles * The JAR files with code that needs to be shipped to the * cluster. If the program uses user-defined functions, @@ -746,9 +746,9 @@ public static StreamExecutionEnvironment createRemoteEnvironment(String host, in * @return A remote environment that executes the program on a cluster. */ public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port, - int degreeOfParallelism, String... jarFiles) { + int parallelism, String... jarFiles) { currentEnvironment = new RemoteStreamEnvironment(host, port, jarFiles); - currentEnvironment.setDegreeOfParallelism(degreeOfParallelism); + currentEnvironment.setParallelism(parallelism); return currentEnvironment; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java index 1cff7e75780d9..2cf5cc2fecc46 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java @@ -32,13 +32,13 @@ protected StreamPlanEnvironment(ExecutionEnvironment env) { super(); this.env = env; - int dop = env.getDegreeOfParallelism(); - if (dop > 0) { - setDegreeOfParallelism(dop); + int parallelism = env.getParallelism(); + if (parallelism > 0) { + setParallelism(parallelism); } else { - setDegreeOfParallelism(GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY, - ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE)); + setParallelism(GlobalConfiguration.getInteger( + ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM)); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java index 0fa37acefccec..e112bbf6bf4c3 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java @@ -68,7 +68,7 @@ public void open(Configuration parameters) throws Exception { // get the target stream stream = target == STD_OUT ? System.out : System.err; - // set the prefix if we have a >1 DOP + // set the prefix if we have a >1 parallelism prefix = (context.getNumberOfParallelSubtasks() > 1) ? ((context.getIndexOfThisSubtask() + 1) + "> ") : null; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java index ec8c226351009..0a423ccf97cd5 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java @@ -26,7 +26,7 @@ * Interface for a stream data source. * *

    Sources implementing this specific interface are executed with - * degree of parallelism 1. To execute your sources in parallel + * parallelism 1. To execute your sources in parallel * see {@link ParallelSourceFunction}.

    * * @param The type of the records produced by this source. diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java index 691b111ade4ae..d04e7e6218758 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java @@ -37,12 +37,12 @@ public class ClusterUtil { * * @param jobGraph * jobGraph - * @param degreeOfParallelism + * @param parallelism * numberOfTaskTrackers * @param memorySize * memorySize */ - public static void runOnMiniCluster(JobGraph jobGraph, int degreeOfParallelism, long memorySize) + public static void runOnMiniCluster(JobGraph jobGraph, int parallelism, long memorySize) throws Exception { Configuration configuration = jobGraph.getJobConfiguration(); @@ -50,7 +50,7 @@ public static void runOnMiniCluster(JobGraph jobGraph, int degreeOfParallelism, LocalFlinkMiniCluster exec = null; configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize); - configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, degreeOfParallelism); + configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism); if (LOG.isInfoEnabled()) { LOG.info("Running on mini cluster"); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java index d85384650381a..0446b617e74d9 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java @@ -27,11 +27,11 @@ public abstract class StreamingProgramTestBase extends AbstractTestBase { - private static final int DEFAULT_DEGREE_OF_PARALLELISM = 4; + private static final int DEFAULT_PARALLELISM = 4; private JobExecutionResult latestExecutionResult; - private int degreeOfParallelism = DEFAULT_DEGREE_OF_PARALLELISM; + private int parallelism = DEFAULT_PARALLELISM; public StreamingProgramTestBase() { @@ -40,16 +40,16 @@ public StreamingProgramTestBase() { public StreamingProgramTestBase(Configuration config) { super(config); - setTaskManagerNumSlots(degreeOfParallelism); + setTaskManagerNumSlots(parallelism); } - public void setDegreeOfParallelism(int degreeOfParallelism) { - this.degreeOfParallelism = degreeOfParallelism; - setTaskManagerNumSlots(degreeOfParallelism); + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + setTaskManagerNumSlots(parallelism); } - public int getDegreeOfParallelism() { - return degreeOfParallelism; + public int getParallelism() { + return parallelism; } public JobExecutionResult getLatestExecutionResult() { @@ -86,7 +86,7 @@ public void testJobWithoutObjectReuse() throws Exception { } // prepare the test environment - TestStreamEnvironment env = new TestStreamEnvironment(this.executor, this.degreeOfParallelism); + TestStreamEnvironment env = new TestStreamEnvironment(this.executor, this.parallelism); env.setAsContext(); // call the test program diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java index 6e0821d1d4fe2..5e785f9f1a9b4 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java @@ -38,15 +38,15 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment { private ForkableFlinkMiniCluster executor; private boolean internalExecutor; - public TestStreamEnvironment(int degreeOfParallelism, long memorySize){ - setDegreeOfParallelism(degreeOfParallelism); + public TestStreamEnvironment(int parallelism, long memorySize){ + setParallelism(parallelism); this.memorySize = memorySize; internalExecutor = true; } - public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int dop){ + public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism){ this.executor = executor; - setDefaultLocalParallelism(dop); + setDefaultLocalParallelism(parallelism); } @Override @@ -62,7 +62,7 @@ public void execute(String jobName) throws Exception { Configuration configuration = jobGraph.getJobConfiguration(); configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, - getDegreeOfParallelism()); + getParallelism()); configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize); executor = new ForkableFlinkMiniCluster(configuration); diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 3dc54d62645da..3d7bf1df53784 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -56,11 +56,11 @@ class DataStream[T](javaStream: JavaStream[T]) { def getJavaStream: JavaStream[T] = javaStream /** - * Sets the degree of parallelism of this operation. This must be greater than 1. + * Sets the parallelism of this operation. This must be greater than 1. */ - def setParallelism(dop: Int): DataStream[T] = { + def setParallelism(parallelism: Int): DataStream[T] = { javaStream match { - case ds: SingleOutputStreamOperator[_, _] => ds.setParallelism(dop) + case ds: SingleOutputStreamOperator[_, _] => ds.setParallelism(parallelism) case _ => throw new UnsupportedOperationException("Operator " + javaStream.toString + " cannot " + "have " + @@ -70,7 +70,7 @@ class DataStream[T](javaStream: JavaStream[T]) { } /** - * Returns the degree of parallelism of this operation. + * Returns the parallelism of this operation. */ def getParallelism: Int = javaStream match { case op: SingleOutputStreamOperator[_, _] => op.getParallelism diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala index 7596e6d297eec..83814897e715d 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala @@ -33,9 +33,9 @@ import org.apache.flink.streaming.api.function.source.FileMonitoringFunction.Wat class StreamExecutionEnvironment(javaEnv: JavaEnv) { /** - * Sets the degree of parallelism (DOP) for operations executed through this environment. - * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with - * x parallel instances. This value can be overridden by specific operations using + * Sets the parallelism for operations executed through this environment. + * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run + * with x parallel instances. This value can be overridden by specific operations using * [[DataStream.setParallelism]]. * @deprecated Please use [[setParallelism]] */ @@ -239,7 +239,7 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { * same type and must be serializable. * * * Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. */ def fromElements[T: ClassTag: TypeInformation](data: T*): DataStream[T] = { val typeInfo = implicitly[TypeInformation[T]] @@ -251,7 +251,7 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { * because the framework may move the elements into the cluster if needed. * * Note that this operation will result in a non-parallel data source, i.e. a data source with - * a degree of parallelism of one. + * a parallelism of one. */ def fromCollection[T: ClassTag: TypeInformation]( data: Seq[T]): DataStream[T] = { @@ -352,16 +352,16 @@ object StreamExecutionEnvironment { * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads). */ def createLocalEnvironment( - degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors()): + parallelism: Int = Runtime.getRuntime.availableProcessors()): StreamExecutionEnvironment = { - new StreamExecutionEnvironment(JavaEnv.createLocalEnvironment(degreeOfParallelism)) + new StreamExecutionEnvironment(JavaEnv.createLocalEnvironment(parallelism)) } /** * Creates a remote execution environment. The remote environment sends (parts of) the program to * a cluster for execution. Note that all file paths used in the program must be accessible from - * the cluster. The execution will use the cluster's default degree of parallelism, unless the - * parallelism is set explicitly via [[StreamExecutionEnvironment.setDegreeOfParallelism()]]. + * the cluster. The execution will use the cluster's default parallelism, unless the + * parallelism is set explicitly via [[StreamExecutionEnvironment.setParallelism()]]. * * @param host The host name or address of the master (JobManager), * where the program should be executed. @@ -380,12 +380,12 @@ object StreamExecutionEnvironment { /** * Creates a remote execution environment. The remote environment sends (parts of) the program * to a cluster for execution. Note that all file paths used in the program must be accessible - * from the cluster. The execution will use the specified degree of parallelism. + * from the cluster. The execution will use the specified parallelism. * * @param host The host name or address of the master (JobManager), * where the program should be executed. * @param port The port of the master (JobManager), where the program should be executed. - * @param degreeOfParallelism The degree of parallelism to use during the execution. + * @param parallelism The parallelism to use during the execution. * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the * program uses * user-defined functions, user-defined input formats, or any libraries, @@ -395,10 +395,10 @@ object StreamExecutionEnvironment { def createRemoteEnvironment( host: String, port: Int, - degreeOfParallelism: Int, + parallelism: Int, jarFiles: String*): StreamExecutionEnvironment = { val javaEnv = JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*) - javaEnv.setDegreeOfParallelism(degreeOfParallelism) + javaEnv.setParallelism(parallelism) new StreamExecutionEnvironment(javaEnv) } } diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java index 9396b66199824..fd328aece009e 100644 --- a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java +++ b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java @@ -41,7 +41,7 @@ /** * This test should logically be located in the 'flink-runtime' tests. However, this project - * has already all dependencies required (flink-java-examples). Also, the DOPOneExecEnv is here. + * has already all dependencies required (flink-java-examples). Also, the ParallelismOneExecEnv is here. */ public class HDFSTest { diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java index e1235b698a1e7..3b2fb7fa1beec 100644 --- a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java +++ b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java @@ -128,7 +128,7 @@ public void testTachyon() { addHDConfToFlinkConf.setString(ConfigConstants.HDFS_DEFAULT_CONFIG, HADOOP_CONFIG_PATH.toString()); GlobalConfiguration.includeConfiguration(addHDConfToFlinkConf); - new DopOneTestEnvironment(); // initialize DOP one + new DopOneTestEnvironment(); // initialize parallelism one WordCount.main(new String[]{input, output}); @@ -157,7 +157,7 @@ static final class DopOneTestEnvironment extends LocalEnvironment { @Override public ExecutionEnvironment createExecutionEnvironment() { LocalEnvironment le = new LocalEnvironment(); - le.setDegreeOfParallelism(1); + le.setParallelism(1); return le; } }); diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java index 788327a94f4f3..435713bc8c836 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java @@ -69,10 +69,10 @@ public abstract class CompilerTestBase { public void setup() { this.dataStats = new DataStatistics(); this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator()); - this.withStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM); + this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator()); - this.noStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM); + this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); } // ------------------------------------------------------------------------ diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java index 6e8689640d578..22140000f0e65 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java @@ -29,11 +29,11 @@ public abstract class JavaProgramTestBase extends AbstractTestBase { - private static final int DEFAULT_DEGREE_OF_PARALLELISM = 4; + private static final int DEFAULT_PARALLELISM = 4; private JobExecutionResult latestExecutionResult; - private int degreeOfParallelism = DEFAULT_DEGREE_OF_PARALLELISM; + private int parallelism = DEFAULT_PARALLELISM; /** * The number of times a test should be repeated. @@ -42,7 +42,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase { * tests repeatedly might help to discover resource leaks, race conditions etc. */ private int numberOfTestRepetitions = 1; - + private boolean isCollectionExecution; public JavaProgramTestBase() { @@ -51,20 +51,20 @@ public JavaProgramTestBase() { public JavaProgramTestBase(Configuration config) { super(config); - setTaskManagerNumSlots(degreeOfParallelism); + setTaskManagerNumSlots(parallelism); } - public void setDegreeOfParallelism(int degreeOfParallelism) { - this.degreeOfParallelism = degreeOfParallelism; - setTaskManagerNumSlots(degreeOfParallelism); + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + setTaskManagerNumSlots(parallelism); } public void setNumberOfTestRepetitions(int numberOfTestRepetitions) { this.numberOfTestRepetitions = numberOfTestRepetitions; } - public int getDegreeOfParallelism() { - return isCollectionExecution ? 1 : degreeOfParallelism; + public int getParallelism() { + return isCollectionExecution ? 1 : parallelism; } public JobExecutionResult getLatestExecutionResult() { @@ -110,7 +110,7 @@ public void testJobWithObjectReuse() throws Exception { } // prepare the test environment - TestEnvironment env = new TestEnvironment(this.executor, this.degreeOfParallelism); + TestEnvironment env = new TestEnvironment(this.executor, this.parallelism); env.getConfig().enableObjectReuse(); env.setAsContext(); @@ -162,7 +162,7 @@ public void testJobWithoutObjectReuse() throws Exception { } // prepare the test environment - TestEnvironment env = new TestEnvironment(this.executor, this.degreeOfParallelism); + TestEnvironment env = new TestEnvironment(this.executor, this.parallelism); env.getConfig().disableObjectReuse(); env.setAsContext(); diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java index 67a4797c14c94..eafe9ad9d32ea 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java @@ -35,7 +35,7 @@ public abstract class RecordAPITestBase extends AbstractTestBase { - protected static final int DOP = 4; + protected static final int parallelism = 4; protected JobExecutionResult jobExecutionResult; @@ -48,7 +48,7 @@ public RecordAPITestBase() { public RecordAPITestBase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java index 44f35e7c54169..02c14344566c5 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java @@ -39,9 +39,9 @@ public class TestEnvironment extends ExecutionEnvironment { protected JobExecutionResult latestResult; - public TestEnvironment(ForkableFlinkMiniCluster executor, int degreeOfParallelism) { + public TestEnvironment(ForkableFlinkMiniCluster executor, int parallelism) { this.executor = executor; - setDegreeOfParallelism(degreeOfParallelism); + setParallelism(parallelism); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java index cd384183617f0..1d7d2066cc678 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java @@ -83,7 +83,7 @@ protected void postSubmit() throws Exception { Assert.assertEquals(Integer.valueOf(3), (Integer) res.getAccumulatorResult("num-lines")); - Assert.assertEquals(Double.valueOf(getDegreeOfParallelism()), (Double)res.getAccumulatorResult("open-close-counter")); + Assert.assertEquals(Double.valueOf(getParallelism()), (Double)res.getAccumulatorResult("open-close-counter")); // Test histogram (words per line distribution) Map dist = Maps.newHashMap(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java index daf5181adbf72..4868aff29b31d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java @@ -95,7 +95,7 @@ protected Plan getTestJob() { MapOperator mp2 = MapOperator.builder(Mp2.class).setBroadcastVariable("z", mp1).input(jn2).build(); FileDataSink output = new FileDataSink(new ContractITCaseOutputFormat(), resultPath); - output.setDegreeOfParallelism(1); + output.setParallelism(1); output.setInput(mp2); return new Plan(output); diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java index 06c7506406753..41d24b89198ae 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java @@ -40,7 +40,7 @@ public class BroadcastVarInitializationITCase extends JavaProgramTestBase { protected void testProgram() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(4); + env.setParallelism(4); DataSet data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8); diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java index e9374b1cf6b78..34d413326d26e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java @@ -67,7 +67,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase { private static final int NUM_FEATURES = 3; - private static final int DOP = 4; + private static final int parallelism = 4; protected String pointsPath; @@ -76,7 +76,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase { protected String resultPath; public BroadcastVarsNepheleITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -131,7 +131,7 @@ protected void preSubmit() throws Exception { @Override protected JobGraph getJobGraph() throws Exception { - return createJobGraphV1(this.pointsPath, this.modelsPath, this.resultPath, DOP); + return createJobGraphV1(this.pointsPath, this.modelsPath, this.resultPath, parallelism); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java index 8e1340873d553..2406d6ee5c16e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java @@ -60,9 +60,9 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase { private static final int MEMORY_PER_CONSUMER = 2; - private static final int DOP = 4; + private static final int parallelism = 4; - private static final double MEMORY_FRACTION_PER_CONSUMER = (double)MEMORY_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*DOP; + private static final double MEMORY_FRACTION_PER_CONSUMER = (double)MEMORY_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*parallelism; protected String dataPath; protected String clusterPath; @@ -70,7 +70,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase { public KMeansIterativeNepheleITCase() { - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -87,7 +87,7 @@ protected void postSubmit() throws Exception { @Override protected JobGraph getJobGraph() throws Exception { - return createJobGraph(dataPath, clusterPath, this.resultPath, DOP, 20); + return createJobGraph(dataPath, clusterPath, this.resultPath, parallelism, 20); } // ------------------------------------------------------------------------------------------------------------- @@ -252,8 +252,8 @@ private static AbstractJobVertex createReducer(JobGraph jobGraph, int numSubTask return tail; } - private static AbstractJobVertex createSync(JobGraph jobGraph, int numIterations, int dop) { - AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, dop); + private static AbstractJobVertex createSync(JobGraph jobGraph, int numIterations, int parallelism) { + AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism); TaskConfig syncConfig = new TaskConfig(sync.getConfiguration()); syncConfig.setNumberOfIterations(numIterations); syncConfig.setIterationId(ITERATION_ID); diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java index 9bbd282c21d64..36db34d2b3866 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java @@ -33,10 +33,10 @@ @SuppressWarnings("deprecation") public class MapCancelingITCase extends CancellingTestBase { - private static final int DOP = 4; + private static final int parallelism = 4; public MapCancelingITCase() { - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } // @Test @@ -51,7 +51,7 @@ public void testMapCancelling() throws Exception { Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 5 * 1000, 10 * 1000); } @@ -68,7 +68,7 @@ public void testSlowMapCancelling() throws Exception { Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 5 * 1000, 10 * 1000); } @@ -85,7 +85,7 @@ public void testMapWithLongCancellingResponse() throws Exception { Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 10 * 1000, 10 * 1000); } @@ -102,7 +102,7 @@ public void testMapPriorToFirstRecordReading() throws Exception { Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 10 * 1000, 10 * 1000); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java index 8a324a3734c8f..8197e27356426 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java @@ -37,10 +37,10 @@ @SuppressWarnings("deprecation") public class MatchJoinCancelingITCase extends CancellingTestBase { - private static final int DOP = 4; + private static final int parallelism = 4; public MatchJoinCancelingITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } // --------------- Test Sort Matches that are canceled while still reading / sorting ----------------- @@ -60,7 +60,7 @@ public void testCancelSortMatchWhileReadingSlowInputs() throws Exception { GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 3000, 10*1000); } @@ -81,7 +81,7 @@ public void testCancelSortMatchWhileReadingFastInputs() throws Exception { GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 5000, 10*1000); } @@ -102,7 +102,7 @@ public void testCancelSortMatchPriorToFirstRecordReading() throws Exception { GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 5000); @@ -129,7 +129,7 @@ public void testCancelSortMatchWhileDoingHeavySorting() throws Exception { GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 30 * 1000, 30 * 1000); } @@ -157,7 +157,7 @@ public void testCancelSortMatchWhileJoining() throws Exception { GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 10 * 1000, 20 * 1000); } @@ -183,7 +183,7 @@ public void testCancelSortMatchWithLongCancellingResponse() throws Exception { GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink"); Plan p = new Plan(sink); - p.setDefaultParallelism(DOP); + p.setDefaultParallelism(parallelism); runAndCancelJob(p, 10 * 1000, 10 * 1000); } @@ -191,7 +191,7 @@ public void testCancelSortMatchWithLongCancellingResponse() throws Exception { // -------------------------------------- Test System corner cases --------------------------------- // @Test - public void testCancelSortMatchWithHighDOP() throws Exception { + public void testCancelSortMatchWithHighparallelism() throws Exception { GenericDataSource source1 = new GenericDataSource(new InfiniteIntegerInputFormat(), "Source 1"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java index 53b70ad6f2eca..62e2893fe4a70 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java @@ -30,7 +30,7 @@ public class LocalExecutorITCase { - private static final int DOP = 4; + private static final int parallelism = 4; @Test public void testLocalExecutorWithWordCount() { @@ -50,11 +50,11 @@ public void testLocalExecutorWithWordCount() { LocalExecutor executor = new LocalExecutor(); executor.setDefaultOverwriteFiles(true); - executor.setTaskManagerNumSlots(DOP); + executor.setTaskManagerNumSlots(parallelism); executor.setPrintStatusDuringExecution(false); executor.start(); - executor.executePlan(wc.getPlan(Integer.valueOf(DOP).toString(), inFile.toURI().toString(), + executor.executePlan(wc.getPlan(Integer.valueOf(parallelism).toString(), inFile.toURI().toString(), outFile.toURI().toString())); executor.stop(); } catch (Exception e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java index a69b5c210d57f..5c1c500585cfe 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java @@ -28,7 +28,7 @@ public class WordCountITCase extends JavaProgramTestBase { protected String resultPath; public WordCountITCase(){ - setDegreeOfParallelism(4); + setParallelism(4); setNumTaskManagers(2); setTaskManagerNumSlots(2); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java index 9021c6a3d040c..0d2c469683eda 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java @@ -42,7 +42,7 @@ @SuppressWarnings("deprecation") public class TaskFailureITCase extends FailingTestBase { - private static final int DOP = 4; + private static final int parallelism = 4; // input for map tasks private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" + @@ -57,7 +57,7 @@ public class TaskFailureITCase extends FailingTestBase { private String resultPath; public TaskFailureITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -85,7 +85,7 @@ protected JobGraph getFailingJobGraph() throws Exception { // generate plan Plan plan = new Plan(output); - plan.setDefaultParallelism(DOP); + plan.setDefaultParallelism(parallelism); // optimize and compile plan Optimizer pc = new Optimizer(new DataStatistics()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java index 0a4673a736f36..9023a1f4fb347 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java @@ -58,7 +58,7 @@ public class CoGroupConnectedComponentsITCase extends RecordAPITestBase { protected String resultPath; public CoGroupConnectedComponentsITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -71,7 +71,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { - return getPlan(DOP, verticesPath, edgesPath, resultPath, 100); + return getPlan(parallelism, verticesPath, edgesPath, resultPath, 100); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java index f6ada63a64b0d..df3c00d32c3a7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java @@ -41,7 +41,7 @@ public class ConnectedComponentsITCase extends RecordAPITestBase { protected String resultPath; public ConnectedComponentsITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -55,7 +55,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { WorksetConnectedComponents cc = new WorksetConnectedComponents(); - return cc.getPlan(Integer.valueOf(DOP).toString(), verticesPath, edgesPath, resultPath, "100"); + return cc.getPlan(Integer.valueOf(parallelism).toString(), verticesPath, edgesPath, resultPath, "100"); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java index 0b1e37239d43d..d5d150d4d61d3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java @@ -65,7 +65,7 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends RecordAPITestBa public ConnectedComponentsWithDeferredUpdateITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -78,7 +78,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { boolean extraMapper = config.getBoolean("ExtraMapper", false); - return getPlan(DOP, verticesPath, edgesPath, resultPath, 100, extraMapper); + return getPlan(parallelism, verticesPath, edgesPath, resultPath, 100, extraMapper); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java index 33650c51e95f5..b97d0ad70f7e2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java @@ -60,7 +60,7 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends RecordAPITest protected String resultPath; public ConnectedComponentsWithSolutionSetFirstITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -73,7 +73,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { - return getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput(DOP, verticesPath, edgesPath, + return getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput(parallelism, verticesPath, edgesPath, resultPath, 100); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java index 39617b4a3742b..e2d095df4e4e6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java @@ -40,7 +40,7 @@ protected void preSubmit() throws Exception { protected Plan getTestJob() { DanglingPageRank pr = new DanglingPageRank(); Plan plan = pr.getPlan( - String.valueOf(DOP), + String.valueOf(parallelism), pagesPath, edgesPath, resultPath, diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java index 7f5015e2ed8c8..caa9d37b00439 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java @@ -40,7 +40,7 @@ public class DeltaIterationNotDependingOnSolutionSetITCase extends JavaProgramTe protected void testProgram() throws Exception { try { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> input = env.generateSequence(0, 9).map(new Duplicator()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java index cf59a3f929cb0..bf459c613516a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java @@ -42,7 +42,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { - String[] params = { String.valueOf(DOP) , verticesPath, edgesPath, resultPath, "3" }; + String[] params = { String.valueOf(parallelism) , verticesPath, edgesPath, resultPath, "3" }; WorksetConnectedComponents cc = new WorksetConnectedComponents(); return cc.getPlan(params); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java index 15079ec7e01d9..0635fe5c0a4cf 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java @@ -43,7 +43,7 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase { private static final int MAX_ITERATIONS = 20; - private static final int DOP = 1; + private static final int parallelism = 1; protected static List> verticesInput = new ArrayList>(); protected static List> edgesInput = new ArrayList>(); @@ -51,7 +51,7 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase { private String expectedResult; public DependencyConnectedComponentsITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -113,7 +113,7 @@ private static class DependencyConnectedComponentsProgram { public static String runProgram(String resultPath) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet> initialSolutionSet = env.fromCollection(verticesInput); DataSet> edges = env.fromCollection(edgesInput); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java index 7b6cf11654a12..0915a42a8031c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java @@ -47,7 +47,7 @@ public class IterationTerminationWithTerminationTail extends RecordAPITestBase { protected String resultPath; public IterationTerminationWithTerminationTail(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -63,7 +63,7 @@ protected void postSubmit() throws Exception { @Override protected Plan getTestJob() { - return getTestPlanPlan(DOP, dataPath, resultPath); + return getTestPlanPlan(parallelism, dataPath, resultPath); } private static Plan getTestPlanPlan(int numSubTasks, String input, String output) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java index f05b1c25cb60a..3ce021bf1672e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java @@ -47,7 +47,7 @@ public class IterationTerminationWithTwoTails extends RecordAPITestBase { protected String resultPath; public IterationTerminationWithTwoTails(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -63,7 +63,7 @@ protected void postSubmit() throws Exception { @Override protected Plan getTestJob() { - return getTestPlanPlan(DOP, dataPath, resultPath); + return getTestPlanPlan(parallelism, dataPath, resultPath); } private static Plan getTestPlanPlan(int numSubTasks, String input, String output) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java index 3116a9df06a5e..cb16c154a2744 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java @@ -61,7 +61,7 @@ protected void postSubmit() throws Exception { @Override protected Plan getTestJob() { - Plan plan = getTestPlanPlan(DOP, dataPath, resultPath); + Plan plan = getTestPlanPlan(parallelism, dataPath, resultPath); return plan; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java index f626b75f154b8..c11c9eab3460f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java @@ -52,7 +52,7 @@ public class IterationWithChainingITCase extends RecordAPITestBase { public IterationWithChainingITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -74,7 +74,7 @@ protected Plan getTestJob() { @Parameters public static Collection getConfigurations() { Configuration config1 = new Configuration(); - config1.setInteger("ChainedMapperITCase#NoSubtasks", DOP); + config1.setInteger("ChainedMapperITCase#NoSubtasks", parallelism); return toParameterList(config1); } @@ -103,7 +103,7 @@ public void reduce(Iterator it, Collector out) { static Plan getTestPlan(int numSubTasks, String input, String output) { FileDataSource initialInput = new FileDataSource(new PointInFormat(), input, "Input"); - initialInput.setDegreeOfParallelism(1); + initialInput.setParallelism(1); BulkIteration iteration = new BulkIteration("Loop"); iteration.setInput(initialInput); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java index 4cd72fd936811..ac3659a8193e7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java @@ -35,7 +35,7 @@ public class IterativeKMeansITCase extends RecordAPITestBase { protected String resultPath; public IterativeKMeansITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -48,7 +48,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { KMeansBroadcast kmi = new KMeansBroadcast(); - return kmi.getPlan(String.valueOf(DOP), dataPath, clusterPath, resultPath, "20"); + return kmi.getPlan(String.valueOf(parallelism), dataPath, clusterPath, resultPath, "20"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java index 24910285a82b7..fcf43df0cbb51 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java @@ -35,7 +35,7 @@ public class KMeansITCase extends RecordAPITestBase { protected String resultPath; public KMeansITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -48,7 +48,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { KMeansBroadcast kmi = new KMeansBroadcast(); - return kmi.getPlan(String.valueOf(DOP), dataPath, clusterPath, resultPath, "20"); + return kmi.getPlan(String.valueOf(parallelism), dataPath, clusterPath, resultPath, "20"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java index 450b360df3726..fa13656b63997 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java @@ -59,7 +59,7 @@ // // public LineRankITCase(Configuration config) { // super(config); -// setTaskManagerNumSlots(DOP); +// setTaskManagerNumSlots(parallelism); // } // // @Override @@ -85,7 +85,7 @@ // @Parameters // public static Collection getConfigurations() { // Configuration config1 = new Configuration(); -// config1.setInteger("NumSubtasks", DOP); +// config1.setInteger("NumSubtasks", parallelism); // config1.setInteger("NumIterations", 5); // return toParameterList(config1); // } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java index 8e42dd79eeac6..946d89b55a527 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java @@ -43,7 +43,7 @@ protected void preSubmit() throws Exception { protected Plan getTestJob() { SimplePageRank pr = new SimplePageRank(); Plan plan = pr.getPlan( - String.valueOf(DOP), + String.valueOf(parallelism), pagesPath, edgesPath, resultPath, diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java index 5aa6b42a03511..44544d37cbac5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java @@ -51,7 +51,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase { private static final int MAX_ITERATIONS = 20; - private static final int DOP = 2; + private static final int parallelism = 2; private static final String NEGATIVE_ELEMENTS_AGGR = "count.negative.elements"; public AggregatorsITCase(TestExecutionMode mode){ @@ -81,7 +81,7 @@ public void testAggregatorWithoutParameterForIterate() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet initialSolutionSet = CollectionDataSets.getIntegerDataSet(env); IterativeDataSet iteration = initialSolutionSet.iterate(MAX_ITERATIONS); @@ -110,7 +110,7 @@ public void testAggregatorWithParameterForIterate() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet initialSolutionSet = CollectionDataSets.getIntegerDataSet(env); IterativeDataSet iteration = initialSolutionSet.iterate(MAX_ITERATIONS); @@ -139,7 +139,7 @@ public void testConvergenceCriterionWithParameterForIterate() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet initialSolutionSet = CollectionDataSets.getIntegerDataSet(env); IterativeDataSet iteration = initialSolutionSet.iterate(MAX_ITERATIONS); @@ -168,7 +168,7 @@ public void testAggregatorWithoutParameterForIterateDelta() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet> initialSolutionSet = CollectionDataSets.getIntegerDataSet(env).map(new TupleMakerMap()); @@ -202,7 +202,7 @@ public void testAggregatorWithParameterForIterateDelta() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet> initialSolutionSet = CollectionDataSets.getIntegerDataSet(env).map(new TupleMakerMap()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java index faaa541139193..8bf50de139115 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java @@ -41,7 +41,7 @@ public class ConnectedComponentsWithParametrizableAggregatorITCase extends JavaProgramTestBase { private static final int MAX_ITERATIONS = 5; - private static final int DOP = 1; + private static final int parallelism = 1; protected static List> verticesInput = new ArrayList>(); protected static List> edgesInput = new ArrayList>(); @@ -118,7 +118,7 @@ private static class ConnectedComponentsWithAggregatorProgram { public static String runProgram(String resultPath) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet> initialSolutionSet = env.fromCollection(verticesInput); DataSet> edges = env.fromCollection(edgesInput); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java index 4d890e9d9a7cb..e616a2b952dad 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java @@ -44,7 +44,7 @@ public class ConnectedComponentsWithParametrizableConvergenceITCase extends JavaProgramTestBase { private static final int MAX_ITERATIONS = 10; - private static final int DOP = 1; + private static final int parallelism = 1; protected static List> verticesInput = new ArrayList>(); protected static List> edgesInput = new ArrayList>(); @@ -111,7 +111,7 @@ private static class ConnectedComponentsWithConvergenceProgram { public static String runProgram(String resultPath) throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(DOP); + env.setParallelism(parallelism); DataSet> initialSolutionSet = env.fromCollection(verticesInput); DataSet> edges = env.fromCollection(edgesInput); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java index 0e568b664a8b8..7cec1223f2a05 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java @@ -89,9 +89,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase { private static final long MEM_PER_CONSUMER = 3; - private static final int DOP = 4; + private static final int parallelism = 4; - private static final double MEM_FRAC_PER_CONSUMER = (double)MEM_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*DOP; + private static final double MEM_FRAC_PER_CONSUMER = (double)MEM_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*parallelism; protected String verticesPath; @@ -101,7 +101,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase { public ConnectedComponentsNepheleITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Parameters @@ -135,14 +135,14 @@ protected JobGraph getJobGraph() throws Exception { int type = config.getInteger("testcase", 0); switch (type) { case 1: - return createJobGraphUnifiedTails(verticesPath, edgesPath, resultPath, DOP, maxIterations); + return createJobGraphUnifiedTails(verticesPath, edgesPath, resultPath, parallelism, maxIterations); case 2: - return createJobGraphSeparateTails(verticesPath, edgesPath, resultPath, DOP, maxIterations); + return createJobGraphSeparateTails(verticesPath, edgesPath, resultPath, parallelism, maxIterations); case 3: - return createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(verticesPath, edgesPath, resultPath, DOP, + return createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(verticesPath, edgesPath, resultPath, parallelism, maxIterations); case 4: - return createJobGraphSolutionSetUpdateAndWorksetTail(verticesPath, edgesPath, resultPath, DOP, + return createJobGraphSolutionSetUpdateAndWorksetTail(verticesPath, edgesPath, resultPath, parallelism, maxIterations); default: throw new RuntimeException("Broken test configuration"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java index f22bc848d4918..516309cd3b56c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java @@ -40,7 +40,7 @@ public class DanglingPageRankNepheleITCase extends RecordAPITestBase { protected String resultPath; public DanglingPageRankNepheleITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -54,7 +54,7 @@ protected void preSubmit() throws Exception { @Override protected JobGraph getJobGraph() throws Exception { String[] parameters = new String[] { - Integer.valueOf(DOP).toString(), + Integer.valueOf(parallelism).toString(), pagesWithRankPath, edgesPath, resultPath, diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java index c4ffd02fd54ce..ba22ce5611e8f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java @@ -29,7 +29,7 @@ public class DanglingPageRankWithCombinerNepheleITCase extends RecordAPITestBase protected String resultPath; public DanglingPageRankWithCombinerNepheleITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -42,7 +42,7 @@ protected void preSubmit() throws Exception { @Override protected JobGraph getJobGraph() throws Exception { String[] parameters = new String[] { - Integer.valueOf(DOP).toString(), + Integer.valueOf(parallelism).toString(), pagesWithRankPath, edgesPath, resultPath, diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java index 2a8e84d9ba403..69ff083fb558e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java @@ -79,7 +79,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase { public IterationWithChainingNepheleITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -99,7 +99,7 @@ protected void postSubmit() throws Exception { @Parameterized.Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("ChainedMapperNepheleITCase#NoSubtasks", DOP); + config.setInteger("ChainedMapperNepheleITCase#NoSubtasks", parallelism); config.setInteger("ChainedMapperNepheleITCase#MaxIterations", 2); return toParameterList(config); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java index 58969c9130866..153a85edd4854 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java @@ -40,20 +40,20 @@ public class JobGraphUtils { private JobGraphUtils() {} public static > InputFormatVertex createInput(T stub, String path, String name, JobGraph graph, - int degreeOfParallelism) + int parallelism) { stub.setFilePath(path); - return createInput(new UserCodeObjectWrapper(stub), name, graph, degreeOfParallelism); + return createInput(new UserCodeObjectWrapper(stub), name, graph, parallelism); } private static > InputFormatVertex createInput(UserCodeWrapper stub, String name, JobGraph graph, - int degreeOfParallelism) + int parallelism) { InputFormatVertex inputVertex = new InputFormatVertex(name); graph.addVertex(inputVertex); inputVertex.setInvokableClass(DataSourceTask.class); - inputVertex.setParallelism(degreeOfParallelism); + inputVertex.setParallelism(parallelism); TaskConfig inputConfig = new TaskConfig(inputVertex.getConfiguration()); inputConfig.setStubWrapper(stub); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java index 070550034744d..8801dd61817dc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java @@ -96,7 +96,7 @@ public class CustomCompensatableDanglingPageRank { public static JobGraph getJobGraph(String[] args) throws Exception { - int degreeOfParallelism = 2; + int parallelism = 2; String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank"; String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + // "test-inputs/danglingpagerank/adjacencylists"; @@ -114,7 +114,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { double messageLoss = 0.75; if (args.length >= 14) { - degreeOfParallelism = Integer.parseInt(args[0]); + parallelism = Integer.parseInt(args[0]); pageWithRankInputPath = args[1]; adjacencyListInputPath = args[2]; outputPath = args[3]; @@ -138,7 +138,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // page rank input InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(), - pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism); + pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism); TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration()); pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0); @@ -147,7 +147,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // edges as adjacency list InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(), - adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism); + adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism); TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration()); adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer); @@ -155,7 +155,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the head --------------------- AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, - degreeOfParallelism); + parallelism); TaskConfig headConfig = new TaskConfig(head.getConfiguration()); headConfig.setIterationId(ITERATION_ID); @@ -200,7 +200,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the join --------------------- AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, - "IterationIntermediate", jobGraph, degreeOfParallelism); + "IterationIntermediate", jobGraph, parallelism); TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration()); intermediateConfig.setIterationId(ITERATION_ID); // intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class); @@ -228,7 +228,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // ---------------- the tail (co group) -------------------- AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph, - degreeOfParallelism); + parallelism); TaskConfig tailConfig = new TaskConfig(tail.getConfiguration()); tailConfig.setIterationId(ITERATION_ID); tailConfig.setIsWorksetUpdate(); @@ -264,7 +264,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the output --------------------- - OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism); + OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism); TaskConfig outputConfig = new TaskConfig(output.getConfiguration()); outputConfig.addInputToGroup(0); outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0); @@ -273,7 +273,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the auxiliaries --------------------- - AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism); + AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism); TaskConfig syncConfig = new TaskConfig(sync.getConfiguration()); syncConfig.setNumberOfIterations(numIterations); syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator()); @@ -292,7 +292,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE); JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL); tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); - tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism); + tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism); JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java index 7bc300fb28a7b..6f19c03c713fb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java @@ -97,7 +97,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner { public static JobGraph getJobGraph(String[] args) throws Exception { - int degreeOfParallelism = 2; + int parallelism = 2; String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank"; String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + // "test-inputs/danglingpagerank/adjacencylists"; @@ -114,7 +114,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { double messageLoss = 0.75; if (args.length >= 14) { - degreeOfParallelism = Integer.parseInt(args[0]); + parallelism = Integer.parseInt(args[0]); pageWithRankInputPath = args[1]; adjacencyListInputPath = args[2]; outputPath = args[3]; @@ -138,7 +138,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // page rank input InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(), - pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism); + pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism); TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration()); pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0); @@ -147,7 +147,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // edges as adjacency list InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(), - adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism); + adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism); TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration()); adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer); @@ -155,7 +155,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the head --------------------- AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, - degreeOfParallelism); + parallelism); TaskConfig headConfig = new TaskConfig(head.getConfiguration()); headConfig.setIterationId(ITERATION_ID); @@ -200,7 +200,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the join --------------------- AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, - "IterationIntermediate", jobGraph, degreeOfParallelism); + "IterationIntermediate", jobGraph, parallelism); TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration()); intermediateConfig.setIterationId(ITERATION_ID); // intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class); @@ -241,7 +241,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // ---------------- the tail (co group) -------------------- AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph, - degreeOfParallelism); + parallelism); TaskConfig tailConfig = new TaskConfig(tail.getConfiguration()); tailConfig.setIterationId(ITERATION_ID); tailConfig.setIsWorksetUpdate(); @@ -278,7 +278,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the output --------------------- - OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism); + OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism); TaskConfig outputConfig = new TaskConfig(output.getConfiguration()); outputConfig.addInputToGroup(0); outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0); @@ -287,7 +287,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the auxiliaries --------------------- - AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism); + AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism); TaskConfig syncConfig = new TaskConfig(sync.getConfiguration()); syncConfig.setNumberOfIterations(numIterations); syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator()); @@ -306,7 +306,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE); JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL); tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); - tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism); + tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism); JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java index f06f723558510..8216ccbb5ccc6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java @@ -77,7 +77,7 @@ public class CompensatableDanglingPageRank { public static JobGraph getJobGraph(String[] args) throws Exception { - int degreeOfParallelism = 2; + int parallelism = 2; String pageWithRankInputPath = ""; // "file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank"; String adjacencyListInputPath = ""; // "file://" + PlayConstants.PLAY_DIR + // "test-inputs/danglingpagerank/adjacencylists"; @@ -95,7 +95,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { double messageLoss = 0.75; if (args.length >= 15) { - degreeOfParallelism = Integer.parseInt(args[0]); + parallelism = Integer.parseInt(args[0]); pageWithRankInputPath = args[1]; adjacencyListInputPath = args[2]; outputPath = args[3]; @@ -119,7 +119,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // page rank input InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(), - pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism); + pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism); TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration()); pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); pageWithRankInputConfig.setOutputComparator(fieldZeroComparator, 0); @@ -128,14 +128,14 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // edges as adjacency list InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(), - adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism); + adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism); TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration()); adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); adjacencyListInputConfig.setOutputSerializer(recSerializer); adjacencyListInputConfig.setOutputComparator(fieldZeroComparator, 0); // --------------- the head --------------------- - AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, degreeOfParallelism); + AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, parallelism); TaskConfig headConfig = new TaskConfig(head.getConfiguration()); headConfig.setIterationId(ITERATION_ID); @@ -179,7 +179,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the join --------------------- - AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, "IterationIntermediate", jobGraph, degreeOfParallelism); + AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, "IterationIntermediate", jobGraph, parallelism); TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration()); intermediateConfig.setIterationId(ITERATION_ID); // intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class); @@ -207,7 +207,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // ---------------- the tail (co group) -------------------- AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph, - degreeOfParallelism); + parallelism); TaskConfig tailConfig = new TaskConfig(tail.getConfiguration()); tailConfig.setIterationId(ITERATION_ID); tailConfig.setIsWorksetUpdate(); @@ -244,7 +244,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the output --------------------- - OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism); + OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism); TaskConfig outputConfig = new TaskConfig(output.getConfiguration()); outputConfig.addInputToGroup(0); outputConfig.setInputSerializer(recSerializer, 0); @@ -253,7 +253,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { // --------------- the auxiliaries --------------------- - AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism); + AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism); TaskConfig syncConfig = new TaskConfig(sync.getConfiguration()); syncConfig.setNumberOfIterations(numIterations); syncConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator()); @@ -272,7 +272,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception { JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE); JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL); tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); - tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism); + tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism); JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java index dbf4798cd14b8..5dc38677d1d0b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java @@ -61,7 +61,7 @@ public void before() throws Exception{ } @Test - public void testIntSortingDOP1() throws Exception { + public void testIntSortingParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -76,7 +76,7 @@ public void testIntSortingDOP1() throws Exception { } @Test - public void testStringSortingDOP1() throws Exception { + public void testStringSortingParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -98,7 +98,7 @@ public void testStringSortingDOP1() throws Exception { } @Test - public void testTupleSortingSingleAscDOP1() throws Exception { + public void testTupleSortingSingleAscParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -134,7 +134,7 @@ public void testTupleSortingSingleAscDOP1() throws Exception { } @Test - public void testTupleSortingSingleDescDOP1() throws Exception { + public void testTupleSortingSingleDescParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -169,7 +169,7 @@ public void testTupleSortingSingleDescDOP1() throws Exception { } @Test - public void testTupleSortingDualDOP1() throws Exception { + public void testTupleSortingDualParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -204,7 +204,7 @@ public void testTupleSortingDualDOP1() throws Exception { } @Test - public void testTupleSortingNestedDOP1() throws Exception { + public void testTupleSortingNestedParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -230,7 +230,7 @@ public void testTupleSortingNestedDOP1() throws Exception { } @Test - public void testTupleSortingNestedDOP1_2() throws Exception { + public void testTupleSortingNestedParallelism1_2() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -256,7 +256,7 @@ public void testTupleSortingNestedDOP1_2() throws Exception { } @Test - public void testPojoSortingSingleDOP1() throws Exception { + public void testPojoSortingSingleParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -279,7 +279,7 @@ public void testPojoSortingSingleDOP1() throws Exception { } @Test - public void testPojoSortingDualDOP1() throws Exception { + public void testPojoSortingDualParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -306,7 +306,7 @@ public void testPojoSortingDualDOP1() throws Exception { } @Test - public void testPojoSortingNestedDOP1() throws Exception { + public void testPojoSortingNestedParallelism1() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -333,7 +333,7 @@ public void testPojoSortingNestedDOP1() throws Exception { } @Test - public void testSortingDOP4() throws Exception { + public void testSortingParallelism4() throws Exception { final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java index 494f3546dc09b..2a97c607f8336 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java @@ -321,12 +321,12 @@ public void combine(Iterable> values, Collector> ds = CollectionDataSets.get3TupleDataSet(env); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java index e5f91b4209b63..c5067f9d102df 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java @@ -138,7 +138,7 @@ public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSort */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds. @@ -346,7 +346,7 @@ public void testCorrectnessOfGroupReduceOnTuplesWithCombine() throws Exception { org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION); final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(2); // important because it determines how often the combiner is called + env.setParallelism(2); // important because it determines how often the combiner is called DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds. @@ -394,7 +394,7 @@ public void testCorrectnessOfGroupreduceWithDescendingGroupSort() throws Excepti * check correctness of groupReduce with descending group sort */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds. @@ -456,7 +456,7 @@ public void testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting( */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds. @@ -590,7 +590,7 @@ public void testStringBasedDefinitionOnGroupSort() throws Exception { * check correctness of groupReduce with descending group sort */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds. @@ -613,7 +613,7 @@ public void testIntBasedDefinitionOnGroupSortForFullNestedTuple() throws Excepti * Test int-based definition on group sort, for (full) nested Tuple */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); DataSet reduceDs = ds.groupBy("f1").sortGroup(0, Order.DESCENDING).reduceGroup(new NestedTupleReducer()); @@ -631,7 +631,7 @@ public void testIntBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exce * Test int-based definition on group sort, for (partial) nested Tuple ASC */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); // f0.f0 is first integer @@ -653,7 +653,7 @@ public void testStringBasedDefinitionOnGroupSortForPartialNestedTuple() throws E * Test string-based definition on group sort, for (partial) nested Tuple DESC */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); // f0.f0 is first integer @@ -672,7 +672,7 @@ public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeys() throws Exce * Test string-based definition on group sort, for two grouping keys */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); // f0.f0 is first integer @@ -691,7 +691,7 @@ public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeysWithPojos() th * Test string-based definition on group sort, for two grouping keys with Pojos */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet ds = CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env); // f0.f0 is first integer @@ -711,7 +711,7 @@ public void testTupleKeySelectorGroupSort() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds @@ -830,7 +830,7 @@ public void testTupleKeySelectorSortWithCombine() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet> reduceDs = ds. @@ -870,7 +870,7 @@ public void testTupleKeySelectorSortCombineOnTuple() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> ds = CollectionDataSets.get5TupleDataSet(env); DataSet> reduceDs = ds @@ -915,7 +915,7 @@ public void testGroupingWithPojoContainingMultiplePojos() throws Exception { * Test grouping with pojo containing multiple pojos (was a bug) */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet ds = CollectionDataSets.getPojoWithMultiplePojos(env); @@ -947,7 +947,7 @@ public void testJavaCollectionsWithinPojos() throws Exception { * Test Java collections within pojos ( == test kryo) */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet ds = CollectionDataSets.getPojoWithCollection(env); // f0.f0 is first integer @@ -982,7 +982,7 @@ public void testGroupByGenericType() throws Exception { * Group by generic type */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet ds = CollectionDataSets.getPojoWithCollection(env); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java index 375baeeb13f34..0080fb1845152 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java @@ -527,7 +527,7 @@ public void testSelectingMultipleFieldsUsingExpressionLanguage() throws Exceptio ds1.join(ds2).where("nestedPojo.longNumber", "number", "str").equalTo("f6","f0","f1"); joinDs.writeAsCsv(resultPath); - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + @@ -548,7 +548,7 @@ public void testNestedIntoTuple() throws Exception { ds1.join(ds2).where("nestedPojo.longNumber", "number","nestedTupleWithCustom.f0").equalTo("f6","f0","f2"); joinDs.writeAsCsv(resultPath); - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + @@ -569,7 +569,7 @@ public void testNestedIntoTupleIntoPojo() throws Exception { ds1.join(ds2).where("nestedTupleWithCustom.f0","nestedTupleWithCustom.f1.myInt","nestedTupleWithCustom.f1.myLong").equalTo("f2","f3","f4"); joinDs.writeAsCsv(resultPath); - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" + @@ -590,7 +590,7 @@ public void testNonPojoToVerifyFullTupleKeys() throws Exception { ds1.join(ds2).where(0).equalTo("f0.f0", "f0.f1"); // key is now Tuple2 joinDs.writeAsCsv(resultPath); - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); expected = "((1,1),one),((1,1),one)\n" + @@ -612,7 +612,7 @@ public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception { ds1.join(ds2).where("f0.f0").equalTo("f0.f0"); // key is now Integer from Tuple2 joinDs.writeAsCsv(resultPath); - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); expected = "((1,1),one),((1,1),one)\n" + @@ -633,7 +633,7 @@ public void testFullPojoWithFullTuple() throws Exception { ds1.join(ds2).where("*").equalTo("*"); joinDs.writeAsCsv(resultPath); - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n"+ @@ -655,7 +655,7 @@ public void testNonPojoToVerifyNestedTupleElementSelectionWithFirstKeyFieldGreat ds2.join(ds2).where("f1.f0").equalTo("f0.f0"); joinDs.writeAsCsv(resultPath); - env.setDegreeOfParallelism(1); + env.setParallelism(1); env.execute(); expected = "((1,1,Hi),(1,1,Hi)),((1,1,Hi),(1,1,Hi))\n" + diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java index e1603ca765aab..3637680e0ce67 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java @@ -148,8 +148,8 @@ public void testForcedRebalancing() throws Exception { env.execute(); StringBuilder result = new StringBuilder(); - int numPerPartition = 2220 / env.getDegreeOfParallelism() / 10; - for (int i = 0; i < env.getDegreeOfParallelism(); i++) { + int numPerPartition = 2220 / env.getParallelism() / 10; + for (int i = 0; i < env.getParallelism(); i++) { result.append('(').append(i).append(',').append(numPerPartition).append(")\n"); } @@ -190,13 +190,13 @@ public Tuple2 map(Tuple2 value) throws Excep } @Test - public void testHashPartitionByKeyFieldAndDifferentDOP() throws Exception { + public void testHashPartitionByKeyFieldAndDifferentParallelism() throws Exception { /* - * Test hash partition by key field and different DOP + * Test hash partition by key field and different parallelism */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(3); + env.setParallelism(3); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); DataSet uniqLongs = ds @@ -221,7 +221,7 @@ public void testHashPartitionWithKeyExpression() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(3); + env.setParallelism(3); DataSet ds = CollectionDataSets.getDuplicatePojoDataSet(env); DataSet uniqLongs = ds diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java index 4bba5583168c7..d961f3ae4a978 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java @@ -72,7 +72,7 @@ public void testSortPartitionByKeyField() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(4); + env.setParallelism(4); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); ds @@ -94,7 +94,7 @@ public void testSortPartitionByTwoKeyFields() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(2); + env.setParallelism(2); DataSet> ds = CollectionDataSets.get5TupleDataSet(env); ds @@ -117,7 +117,7 @@ public void testSortPartitionByFieldExpression() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(4); + env.setParallelism(4); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); ds @@ -139,7 +139,7 @@ public void testSortPartitionByTwoFieldExpressions() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(2); + env.setParallelism(2); DataSet> ds = CollectionDataSets.get5TupleDataSet(env); ds @@ -162,7 +162,7 @@ public void testSortPartitionByNestedFieldExpression() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(3); + env.setParallelism(3); DataSet, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env); ds @@ -185,7 +185,7 @@ public void testSortPartitionPojoByNestedFieldExpression() throws Exception { */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(3); + env.setParallelism(3); DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); ds @@ -202,17 +202,17 @@ public void testSortPartitionPojoByNestedFieldExpression() throws Exception { } @Test - public void testSortPartitionDOPChange() throws Exception { + public void testSortPartitionParallelismChange() throws Exception { /* - * Test sort partition with DOP change + * Test sort partition with parallelism change */ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(3); + env.setParallelism(3); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); ds - .sortPartition(1, Order.DESCENDING).setParallelism(3) // change DOP + .sortPartition(1, Order.DESCENDING).setParallelism(3) // change parallelism .mapPartition(new OrderCheckMapper>(new Tuple3Checker())) .distinct() .writeAsText(resultPath); diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java index 8ddd7bc660ab6..aeab77b0e6314 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java @@ -78,7 +78,7 @@ public void testProgramWithAutoParallelism() { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX); + env.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX); DataSet result = env .createInput(new ParallelismDependentInputFormat()) diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java index c308007d5b146..39a08d242b375 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java @@ -34,7 +34,7 @@ protected void testProgram() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); if (!isCollectionExecution()) { - Assert.assertTrue(env.getDegreeOfParallelism() > 1); + Assert.assertTrue(env.getParallelism() > 1); } env.generateSequence(1, 1000) diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java index 2087b6332296f..e0ebadd2c25fb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java @@ -48,7 +48,7 @@ public void testNullValues() { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet data = env.fromElements("hallo") .map(new MapFunction() { diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java index 220611d452347..be0518692651c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java @@ -146,23 +146,23 @@ protected Plan getTestJob() { FileDataSource input_left = new FileDataSource(new CoGroupTestInFormat(), leftInPath); DelimitedInputFormat.configureDelimitedFormat(input_left) .recordDelimiter('\n'); - input_left.setDegreeOfParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); + input_left.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); FileDataSource input_right = new FileDataSource(new CoGroupTestInFormat(), rightInPath); DelimitedInputFormat.configureDelimitedFormat(input_right) .recordDelimiter('\n'); - input_right.setDegreeOfParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); + input_right.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); CoGroupOperator testCoGrouper = CoGroupOperator.builder(new TestCoGrouper(), StringValue.class, 0, 0) .build(); - testCoGrouper.setDegreeOfParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); + testCoGrouper.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1)); testCoGrouper.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, config.getString("CoGroupTest#LocalStrategy", "")); testCoGrouper.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY, config.getString("CoGroupTest#ShipStrategy", "")); FileDataSink output = new FileDataSink(new CoGroupOutFormat(), resultPath); - output.setDegreeOfParallelism(1); + output.setParallelism(1); output.setInput(testCoGrouper); testCoGrouper.setFirstInput(input_left); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java index f6b4127ccc72f..3fde5a9f81758 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java @@ -122,16 +122,16 @@ protected Plan getTestJob() { new ContractITCaseInputFormat(), leftInPath); DelimitedInputFormat.configureDelimitedFormat(input_left) .recordDelimiter('\n'); - input_left.setDegreeOfParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); + input_left.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); FileDataSource input_right = new FileDataSource( new ContractITCaseInputFormat(), rightInPath); DelimitedInputFormat.configureDelimitedFormat(input_right) .recordDelimiter('\n'); - input_right.setDegreeOfParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); + input_right.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); CrossOperator testCross = CrossOperator.builder(new TestCross()).build(); - testCross.setDegreeOfParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); + testCross.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1)); testCross.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, config.getString("CrossTest#LocalStrategy", "")); if (config.getString("CrossTest#ShipStrategy", "").equals("BROADCAST_FIRST")) { @@ -151,7 +151,7 @@ protected Plan getTestJob() { FileDataSink output = new FileDataSink( new ContractITCaseOutputFormat(), resultPath); - output.setDegreeOfParallelism(1); + output.setParallelism(1); output.setInput(testCross); testCross.setFirstInput(input_left); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java index 02a6e38319268..c2ec55ad4e1b4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java @@ -111,17 +111,17 @@ protected Plan getTestJob() { new ContractITCaseInputFormat(), leftInPath); DelimitedInputFormat.configureDelimitedFormat(input_left) .recordDelimiter('\n'); - input_left.setDegreeOfParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); + input_left.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); FileDataSource input_right = new FileDataSource( new ContractITCaseInputFormat(), rightInPath); DelimitedInputFormat.configureDelimitedFormat(input_right) .recordDelimiter('\n'); - input_right.setDegreeOfParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); + input_right.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); JoinOperator testMatcher = JoinOperator.builder(new TestMatcher(), StringValue.class, 0, 0) .build(); - testMatcher.setDegreeOfParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); + testMatcher.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1)); testMatcher.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, config.getString("MatchTest#LocalStrategy", "")); if (config.getString("MatchTest#ShipStrategy", "").equals("BROADCAST_FIRST")) { @@ -141,7 +141,7 @@ protected Plan getTestJob() { FileDataSink output = new FileDataSink( new ContractITCaseOutputFormat(), resultPath); - output.setDegreeOfParallelism(1); + output.setParallelism(1); output.setInput(testMatcher); testMatcher.setFirstInput(input_left); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java index 0d7bfdf7fc939..28b9501add035 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java @@ -100,14 +100,14 @@ protected Plan getTestJob() { new ContractITCaseInputFormat(), inPath); DelimitedInputFormat.configureDelimitedFormat(input) .recordDelimiter('\n'); - input.setDegreeOfParallelism(config.getInteger("MapTest#NoSubtasks", 1)); + input.setParallelism(config.getInteger("MapTest#NoSubtasks", 1)); MapOperator testMapper = MapOperator.builder(new TestMapper()).build(); - testMapper.setDegreeOfParallelism(config.getInteger("MapTest#NoSubtasks", 1)); + testMapper.setParallelism(config.getInteger("MapTest#NoSubtasks", 1)); FileDataSink output = new FileDataSink( new ContractITCaseOutputFormat(), resultPath); - output.setDegreeOfParallelism(1); + output.setParallelism(1); output.setInput(testMapper); testMapper.setInput(input); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java index 3c8d37202d32c..498a6386fb1f3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java @@ -114,11 +114,11 @@ protected JobGraph getJobGraph() throws Exception { new ContractITCaseInputFormat(), inPath); DelimitedInputFormat.configureDelimitedFormat(input) .recordDelimiter('\n'); - input.setDegreeOfParallelism(config.getInteger("ReduceTest#NoSubtasks", 1)); + input.setParallelism(config.getInteger("ReduceTest#NoSubtasks", 1)); ReduceOperator testReducer = ReduceOperator.builder(new TestReducer(), StringValue.class, 0) .build(); - testReducer.setDegreeOfParallelism(config.getInteger("ReduceTest#NoSubtasks", 1)); + testReducer.setParallelism(config.getInteger("ReduceTest#NoSubtasks", 1)); testReducer.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY, config.getString("ReduceTest#LocalStrategy", "")); testReducer.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY, @@ -126,7 +126,7 @@ protected JobGraph getJobGraph() throws Exception { FileDataSink output = new FileDataSink( new ContractITCaseOutputFormat(), resultPath); - output.setDegreeOfParallelism(1); + output.setParallelism(1); output.setInput(testReducer); testReducer.setInput(input); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java index 481ae2d770a68..b833421b406d5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java @@ -111,20 +111,20 @@ protected Plan getTestJob() { new ContractITCaseInputFormat(), input1Path); DelimitedInputFormat.configureDelimitedFormat(input1) .recordDelimiter('\n'); - input1.setDegreeOfParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); + input1.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); FileDataSource input2 = new FileDataSource( new ContractITCaseInputFormat(), input2Path); DelimitedInputFormat.configureDelimitedFormat(input2) .recordDelimiter('\n'); - input2.setDegreeOfParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); + input2.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); MapOperator testMapper = MapOperator.builder(new TestMapper()).build(); - testMapper.setDegreeOfParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); + testMapper.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1)); FileDataSink output = new FileDataSink( new ContractITCaseOutputFormat(), resultPath); - output.setDegreeOfParallelism(1); + output.setParallelism(1); output.setInput(testMapper); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java index 944de981bbaa8..c214dbd98c1be 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java @@ -50,7 +50,7 @@ public class UnionSinkITCase extends RecordAPITestBase { public UnionSinkITCase(Configuration testConfig) { super(testConfig); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" + @@ -122,7 +122,7 @@ protected JobGraph getJobGraph() throws Exception { output.addInput(testMapper2); Plan plan = new Plan(output); - plan.setDefaultParallelism(DOP); + plan.setDefaultParallelism(parallelism); Optimizer pc = new Optimizer(new DataStatistics()); OptimizedPlan op = pc.compile(plan); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java index 16d25176839e8..86eafe5c1b489 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java @@ -42,12 +42,12 @@ @SuppressWarnings("deprecation") public class CollectionSourceTest extends RecordAPITestBase { - private static final int DOP = 4; + private static final int parallelism = 4; protected String resultPath; public CollectionSourceTest(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } public static class Join extends JoinFunction { @@ -122,7 +122,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { - return getPlan(DOP, resultPath); + return getPlan(parallelism, resultPath); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java index 0a6d29d6c16e8..613de78b787eb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java @@ -40,7 +40,7 @@ public class ComputeEdgeDegreesITCase extends RecordAPITestBase { public ComputeEdgeDegreesITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -64,7 +64,7 @@ protected void postSubmit() throws Exception { @Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("NumSubtasks", DOP); + config.setInteger("NumSubtasks", parallelism); return toParameterList(config); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java index c758f324838b5..6b0f01cc47f30 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java @@ -39,7 +39,7 @@ public class EnumTrianglesOnEdgesWithDegreesITCase extends RecordAPITestBase { public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -65,7 +65,7 @@ protected void postSubmit() throws Exception { @Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("NumSubtasks", DOP); + config.setInteger("NumSubtasks", parallelism); return toParameterList(config); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java index f0a1288cbe250..b96a31c742b30 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java @@ -59,7 +59,7 @@ protected void preSubmit() throws Exception { protected Plan getTestJob() { EnumTrianglesRdfFoaf enumTriangles = new EnumTrianglesRdfFoaf(); return enumTriangles.getPlan( - String.valueOf(config.getInteger("NumSubtasks", DOP)), edgesPath, resultPath); + String.valueOf(config.getInteger("NumSubtasks", parallelism)), edgesPath, resultPath); } @Override @@ -70,7 +70,7 @@ protected void postSubmit() throws Exception { @Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("NumSubtasks", DOP); + config.setInteger("NumSubtasks", parallelism); return toParameterList(config); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java index f23b12944b4f7..15486cb5fbdd1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java @@ -45,7 +45,7 @@ public class GlobalSortingITCase extends RecordAPITestBase { private String sortedRecords; public GlobalSortingITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -86,7 +86,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { GlobalSort globalSort = new GlobalSort(); - return globalSort.getPlan(Integer.valueOf(DOP).toString(), recordsPath, resultPath); + return globalSort.getPlan(Integer.valueOf(parallelism).toString(), recordsPath, resultPath); } @Override @@ -108,7 +108,7 @@ public Plan getPlan(String... args) throws IllegalArgumentException { String output = (args.length > 2 ? args[2] : ""); FileDataSource source = new FileDataSource(CsvInputFormat.class, recordsPath); - source.setDegreeOfParallelism(numSubtasks); + source.setParallelism(numSubtasks); CsvInputFormat.configureRecordFormat(source) .recordDelimiter('\n') .fieldDelimiter('|') @@ -116,7 +116,7 @@ public Plan getPlan(String... args) throws IllegalArgumentException { FileDataSink sink = new FileDataSink(CsvOutputFormat.class, output); - sink.setDegreeOfParallelism(numSubtasks); + sink.setParallelism(numSubtasks); CsvOutputFormat.configureRecordFormat(sink) .recordDelimiter('\n') .fieldDelimiter('|') diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java index b6b8b9d8b4379..e79b8078eda2c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java @@ -53,7 +53,7 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase { private String sortedRecords; public GlobalSortingMixedOrderITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -91,7 +91,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { GlobalSort globalSort = new GlobalSort(); - return globalSort.getPlan(Integer.valueOf(DOP).toString(), recordsPath, resultPath); + return globalSort.getPlan(Integer.valueOf(parallelism).toString(), recordsPath, resultPath); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java index 368f9af837811..c1ad83dc7a33f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java @@ -65,7 +65,7 @@ public class GroupOrderReduceITCase extends RecordAPITestBase { public GroupOrderReduceITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -78,7 +78,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { - int dop = this.config.getInteger("GroupOrderTest#NumSubtasks", 1); + int parallelism = this.config.getInteger("GroupOrderTest#NumSubtasks", 1); @SuppressWarnings("unchecked") CsvInputFormat format = new CsvInputFormat(',', IntValue.class, IntValue.class); @@ -99,7 +99,7 @@ protected Plan getTestJob() { .field(IntValue.class, 1); Plan p = new Plan(sink); - p.setDefaultParallelism(dop); + p.setDefaultParallelism(parallelism); return p; } @@ -110,7 +110,7 @@ protected void postSubmit() throws Exception { @Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("GroupOrderTest#NumSubtasks", DOP); + config.setInteger("GroupOrderTest#NumSubtasks", parallelism); return toParameterList(config); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java index 231f196a75fd5..1a0cae2dc3b7e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java @@ -71,7 +71,7 @@ protected void preSubmit() throws Exception { protected Plan getTestJob() { PairwiseSP a2aSP = new PairwiseSP(); return a2aSP.getPlan( - String.valueOf(config.getInteger("All2AllSPTest#NoSubtasks", DOP)), + String.valueOf(config.getInteger("All2AllSPTest#NoSubtasks", parallelism)), rdfDataPath, resultPath, "true"); @@ -85,7 +85,7 @@ protected void postSubmit() throws Exception { @Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("All2AllSPTest#NoSubtasks", DOP); + config.setInteger("All2AllSPTest#NoSubtasks", parallelism); return toParameterList(config); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java index b8eb4d2a77037..349275cf7e687 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java @@ -205,7 +205,7 @@ protected void postSubmit() throws Exception { @Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("TPCHQuery10Test#NoSubtasks", DOP); + config.setInteger("TPCHQuery10Test#NoSubtasks", parallelism); return toParameterList(config); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java index 461d6c0fff3e9..a0236c2767410 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java @@ -126,7 +126,7 @@ public class TPCHQuery3ITCase extends RecordAPITestBase { public TPCHQuery3ITCase(Configuration config) { super(config); - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -141,7 +141,7 @@ protected Plan getTestJob() { TPCHQuery3 tpch3 = new TPCHQuery3(); return tpch3.getPlan( - String.valueOf(config.getInteger("dop", 1)), + String.valueOf(config.getInteger("parallelism", 1)), ordersPath, lineitemsPath, resultPath); @@ -155,7 +155,7 @@ protected void postSubmit() throws Exception { @Parameters public static Collection getConfigurations() { Configuration config = new Configuration(); - config.setInteger("dop", DOP); + config.setInteger("parallelism", parallelism); return toParameterList(config); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java index 84a39304a9166..3ade96438ff29 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java @@ -128,7 +128,7 @@ public class TPCHQuery3WithUnionITCase extends RecordAPITestBase { private static final String EXPECTED_RESULT = "5|0|147828.97\n" + "66|0|99188.09\n"; public TPCHQuery3WithUnionITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -146,7 +146,7 @@ protected void preSubmit() throws Exception { protected Plan getTestJob() { TPCHQuery3Unioned tpch3 = new TPCHQuery3Unioned(); return tpch3.getPlan( - Integer.valueOf(DOP).toString(), + Integer.valueOf(parallelism).toString(), orders1Path, orders2Path, partJoin1Path, diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java index 8ddfa304f122e..30c1b3e2f308e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java @@ -119,7 +119,7 @@ public class TPCHQuery4ITCase extends RecordAPITestBase { private static final String EXPECTED_RESULT = "1-URGENT|2|\n" + "3-MEDIUM|2|\n" + "4-NOT SPECIFIED|4|"; public TPCHQuery4ITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -132,7 +132,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { TPCHQuery4 tpch4 = new TPCHQuery4(); - return tpch4.getPlan(Integer.valueOf(DOP).toString(), ordersPath, lineitemsPath, resultPath); + return tpch4.getPlan(Integer.valueOf(parallelism).toString(), ordersPath, lineitemsPath, resultPath); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java index c329ce3c91952..f09240088ba75 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java @@ -343,7 +343,7 @@ public class TPCHQuery9ITCase extends RecordAPITestBase { + "IRAN|1996|9672.556\n"; public TPCHQuery9ITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -362,7 +362,7 @@ protected void preSubmit() throws Exception { protected Plan getTestJob() { TPCHQuery9 tpch9 = new TPCHQuery9(); return tpch9.getPlan( - Integer.valueOf(DOP).toString(), + Integer.valueOf(parallelism).toString(), partInputPath, partSuppInputPath, ordersInputPath, diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java index 769120b68824c..2c53ee28a8cfd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java @@ -69,7 +69,7 @@ public class TPCHQueryAsterixITCase extends RecordAPITestBase { "2|FURNITURE\n"; public TPCHQueryAsterixITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -83,7 +83,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { TPCHQueryAsterix tpchBench = new TPCHQueryAsterix(); - return tpchBench.getPlan(Integer.valueOf(DOP).toString(), ordersPath, custPath, resultPath); + return tpchBench.getPlan(Integer.valueOf(parallelism).toString(), ordersPath, custPath, resultPath); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java index 2cb0f8f18e1c8..a45db2ffe4f85 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java @@ -36,7 +36,7 @@ public class TeraSortITCase extends RecordAPITestBase { private String resultPath; public TeraSortITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -49,7 +49,7 @@ protected Plan getTestJob() { String testDataPath = getClass().getResource(INPUT_DATA_FILE).toString(); TeraSort ts = new TeraSort(); - return ts.getPlan(Integer.valueOf(DOP).toString(), testDataPath, resultPath); + return ts.getPlan(Integer.valueOf(parallelism).toString(), testDataPath, resultPath); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java index 109e62deeae37..f69567b7bff5f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java @@ -155,7 +155,7 @@ public class WebLogAnalysisITCase extends RecordAPITestBase { private static final String expected = "87|url_24|39\n" + "59|url_28|41\n"; public WebLogAnalysisITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @Override @@ -169,7 +169,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { WebLogAnalysis relOLAP = new WebLogAnalysis(); - return relOLAP.getPlan(Integer.valueOf(DOP).toString(), docsPath, ranksPath, visitsPath, resultPath); + return relOLAP.getPlan(Integer.valueOf(parallelism).toString(), docsPath, ranksPath, visitsPath, resultPath); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java index 165c2d35c8999..19f3decbb809b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java @@ -30,7 +30,7 @@ public class WordCountITCase extends RecordAPITestBase { protected String resultPath; public WordCountITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -43,7 +43,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { WordCount wc = new WordCount(); - return wc.getPlan(Integer.valueOf(DOP).toString(), textPath, resultPath); + return wc.getPlan(Integer.valueOf(parallelism).toString(), textPath, resultPath); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java index 52c815b071860..4518568250fcc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java @@ -51,7 +51,7 @@ public class WordCountUnionReduceITCase extends RecordAPITestBase { private String outputPath; public WordCountUnionReduceITCase(){ - setTaskManagerNumSlots(DOP); + setTaskManagerNumSlots(parallelism); } @@ -69,7 +69,7 @@ protected void preSubmit() throws Exception { @Override protected Plan getTestJob() { WordCountUnionReduce wc = new WordCountUnionReduce(); - return wc.getPlan(this.inputPath, this.outputPath, DOP); + return wc.getPlan(this.inputPath, this.outputPath, parallelism); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java index c87dd643fef7c..80ba91af66f4e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java @@ -44,7 +44,7 @@ public class DanglingPageRank implements Program, ProgramDescription { public static final String NUM_VERTICES_CONFIG_PARAM = "pageRank.numVertices"; public Plan getPlan(String ... args) { - int dop = 1; + int parallelism = 1; String pageWithRankInputPath = ""; String adjacencyListInputPath = ""; String outputPath = ""; @@ -53,7 +53,7 @@ public Plan getPlan(String ... args) { long numDanglingVertices = 1; if (args.length >= 7) { - dop = Integer.parseInt(args[0]); + parallelism = Integer.parseInt(args[0]); pageWithRankInputPath = args[1]; adjacencyListInputPath = args[2]; outputPath = args[3]; @@ -94,12 +94,12 @@ public Plan getPlan(String ... args) { FileDataSink out = new FileDataSink(new PageWithRankOutFormat(), outputPath, iteration, "Final Ranks"); Plan p = new Plan(out, "Dangling PageRank"); - p.setDefaultParallelism(dop); + p.setDefaultParallelism(parallelism); return p; } @Override public String getDescription() { - return "Parameters: "; + return "Parameters: "; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java index 55e2f57a20716..34d4b60775e99 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java @@ -399,24 +399,24 @@ public Plan getPlan(String... args) { } else { pathsInput = new FileDataSource(new PathInFormat(), paths, "Paths"); } - pathsInput.setDegreeOfParallelism(numSubTasks); + pathsInput.setParallelism(numSubTasks); JoinOperator concatPaths = JoinOperator.builder(new ConcatPaths(), StringValue.class, 0, 1) .name("Concat Paths") .build(); - concatPaths.setDegreeOfParallelism(numSubTasks); + concatPaths.setParallelism(numSubTasks); CoGroupOperator findShortestPaths = CoGroupOperator.builder(new FindShortestPath(), StringValue.class, 0, 0) .keyField(StringValue.class, 1, 1) .name("Find Shortest Paths") .build(); - findShortestPaths.setDegreeOfParallelism(numSubTasks); + findShortestPaths.setParallelism(numSubTasks); FileDataSink result = new FileDataSink(new PathOutFormat(),output, "New Paths"); - result.setDegreeOfParallelism(numSubTasks); + result.setParallelism(numSubTasks); result.setInput(findShortestPaths); findShortestPaths.setFirstInput(pathsInput); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java index 3abf743f7a474..31d992f6b1191 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java @@ -132,7 +132,7 @@ public void join(Record pageWithRank, Record newPageWithRank, Collector // -------------------------------------------------------------------------------------------- public Plan getPlan(String ... args) { - int dop = 1; + int parallelism = 1; String pageWithRankInputPath = ""; String adjacencyListInputPath = ""; String outputPath = ""; @@ -140,7 +140,7 @@ public Plan getPlan(String ... args) { long numVertices = 5; if (args.length >= 6) { - dop = Integer.parseInt(args[0]); + parallelism = Integer.parseInt(args[0]); pageWithRankInputPath = args[1]; adjacencyListInputPath = args[2]; outputPath = args[3]; @@ -183,12 +183,12 @@ public Plan getPlan(String ... args) { FileDataSink out = new FileDataSink(new PageWithRankOutFormat(), outputPath, iteration, "Final Ranks"); Plan p = new Plan(out, "Simple PageRank"); - p.setDefaultParallelism(dop); + p.setDefaultParallelism(parallelism); return p; } @Override public String getDescription() { - return "Parameters: "; + return "Parameters: "; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java index e5f519d43be4e..d528f5d8a4ae9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java @@ -54,7 +54,7 @@ public class KMeansBroadcast implements Program, ProgramDescription { @Override public Plan getPlan(String... args) { // parse job parameters - int degreeOfParallelism = (args.length > 0 ? Integer.parseInt(args[0]) : 1); + int parallelism = (args.length > 0 ? Integer.parseInt(args[0]) : 1); String dataPointInput = (args.length > 1 ? args[1] : ""); String clusterInput = (args.length > 2 ? args[2] : ""); String output = (args.length > 3 ? args[3] : ""); @@ -99,7 +99,7 @@ public Plan getPlan(String... args) { FileDataSink newClusterPoints = new FileDataSink(new PointOutFormat(), output, iter, "New Center Positions"); Plan plan = new Plan(newClusterPoints, "K-Means"); - plan.setDefaultParallelism(degreeOfParallelism); + plan.setDefaultParallelism(parallelism); return plan; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java index 4069f9a161cf4..8d75d4766f079 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java @@ -53,7 +53,7 @@ public Plan getPlan(String... args) { // create DataSourceContract for cluster center input FileDataSource initialClusterPoints = new FileDataSource(new PointInFormat(), clusterInput, "Centers"); - initialClusterPoints.setDegreeOfParallelism(1); + initialClusterPoints.setParallelism(1); BulkIteration iteration = new BulkIteration("K-Means Loop"); iteration.setInput(initialClusterPoints); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java index 74b8f4c74d616..b94880487f236 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java @@ -89,13 +89,13 @@ public Plan getPlan(final String... args) { @SuppressWarnings("unchecked") CsvInputFormat format2 = new CsvInputFormat('|', IntValue.class, IntValue.class); FileDataSource input2 = new FileDataSource(format2, input2Path, "Input 2"); - input2.setDegreeOfParallelism(numSubtasksInput2); + input2.setParallelism(numSubtasksInput2); ReduceOperator aggInput2 = ReduceOperator.builder(DummyReduce.class, IntValue.class, 0) .input(input2) .name("AggLines") .build(); - aggInput2.setDegreeOfParallelism(numSubtasksInput2); + aggInput2.setParallelism(numSubtasksInput2); // create JoinOperator for joining Orders and LineItems JoinOperator joinLiO = JoinOperator.builder(JoinInputs.class, IntValue.class, 0, 0) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java index 3444b479ced1f..d805b92b4cf2a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java @@ -36,7 +36,7 @@ public class TPCHQuery1 implements Program, ProgramDescription { private static final long serialVersionUID = 1L; - private int degreeOfParallelism = 1; + private int parallelism = 1; private String lineItemInputPath; private String outputPath; @@ -45,28 +45,28 @@ public Plan getPlan(String... args) throws IllegalArgumentException { if (args.length != 3) { - this.degreeOfParallelism = 1; + this.parallelism = 1; this.lineItemInputPath = ""; this.outputPath = ""; } else { - this.degreeOfParallelism = Integer.parseInt(args[0]); + this.parallelism = Integer.parseInt(args[0]); this.lineItemInputPath = args[1]; this.outputPath = args[2]; } FileDataSource lineItems = new FileDataSource(new IntTupleDataInFormat(), this.lineItemInputPath, "LineItems"); - lineItems.setDegreeOfParallelism(this.degreeOfParallelism); + lineItems.setParallelism(this.parallelism); FileDataSink result = new FileDataSink(new StringTupleDataOutFormat(), this.outputPath, "Output"); - result.setDegreeOfParallelism(this.degreeOfParallelism); + result.setParallelism(this.parallelism); MapOperator lineItemFilter = MapOperator.builder(new LineItemFilter()) .name("LineItem Filter") .build(); - lineItemFilter.setDegreeOfParallelism(this.degreeOfParallelism); + lineItemFilter.setParallelism(this.parallelism); ReduceOperator groupByReturnFlag = ReduceOperator.builder(new GroupByReturnFlag(), StringValue.class, 0) @@ -82,6 +82,6 @@ public Plan getPlan(String... args) throws IllegalArgumentException { @Override public String getDescription() { - return "Parameters: [dop] [lineitem-input] [output]"; + return "Parameters: [parallelism] [lineitem-input] [output]"; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java index 7a8ffc6842b87..4bb0cdf117b7f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java @@ -273,12 +273,12 @@ public Plan getPlan(String... args) throws IllegalArgumentException { final String nationsPath; final String resultPath; - final int degreeOfParallelism; + final int parallelism; if (args.length < 6) { throw new IllegalArgumentException("Invalid number of parameters"); } else { - degreeOfParallelism = Integer.parseInt(args[0]); + parallelism = Integer.parseInt(args[0]); ordersPath = args[1]; lineitemsPath = args[2]; customersPath = args[3]; @@ -359,7 +359,7 @@ public Plan getPlan(String... args) throws IllegalArgumentException { // return the PACT plan Plan p = new Plan(result, "TPCH Q10"); - p.setDefaultParallelism(degreeOfParallelism); + p.setDefaultParallelism(parallelism); return p; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java index 2103747fecac3..ec3c5b4f2d7f1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java @@ -56,7 +56,7 @@ public class TPCHQuery4 implements Program, ProgramDescription { private static Logger LOG = LoggerFactory.getLogger(TPCHQuery4.class); - private int degreeOfParallelism = 1; + private int parallelism = 1; private String ordersInputPath; private String lineItemInputPath; private String outputPath; @@ -218,40 +218,40 @@ public Plan getPlan(String... args) throws IllegalArgumentException { FileDataSource orders = new FileDataSource(new IntTupleDataInFormat(), this.ordersInputPath, "Orders"); - orders.setDegreeOfParallelism(this.degreeOfParallelism); + orders.setParallelism(this.parallelism); //orders.setOutputContract(UniqueKey.class); FileDataSource lineItems = new FileDataSource(new IntTupleDataInFormat(), this.lineItemInputPath, "LineItems"); - lineItems.setDegreeOfParallelism(this.degreeOfParallelism); + lineItems.setParallelism(this.parallelism); FileDataSink result = new FileDataSink(new StringTupleDataOutFormat(), this.outputPath, "Output"); - result.setDegreeOfParallelism(degreeOfParallelism); + result.setParallelism(parallelism); MapOperator lineFilter = MapOperator.builder(LiFilter.class) .name("LineItemFilter") .build(); - lineFilter.setDegreeOfParallelism(degreeOfParallelism); + lineFilter.setParallelism(parallelism); MapOperator ordersFilter = MapOperator.builder(OFilter.class) .name("OrdersFilter") .build(); - ordersFilter.setDegreeOfParallelism(degreeOfParallelism); + ordersFilter.setParallelism(parallelism); JoinOperator join = JoinOperator.builder(JoinLiO.class, IntValue.class, 0, 0) .name("OrdersLineitemsJoin") .build(); - join.setDegreeOfParallelism(degreeOfParallelism); + join.setParallelism(parallelism); ReduceOperator aggregation = ReduceOperator.builder(CountAgg.class, StringValue.class, 0) .name("AggregateGroupBy") .build(); - aggregation.setDegreeOfParallelism(this.degreeOfParallelism); + aggregation.setParallelism(this.parallelism); lineFilter.setInput(lineItems); ordersFilter.setInput(orders); @@ -269,7 +269,7 @@ public Plan getPlan(String... args) throws IllegalArgumentException { * @param args */ private void setArgs(String[] args) { - this.degreeOfParallelism = Integer.parseInt(args[0]); + this.parallelism = Integer.parseInt(args[0]); this.ordersInputPath = args[1]; this.lineItemInputPath = args[2]; this.outputPath = args[3]; @@ -278,7 +278,7 @@ private void setArgs(String[] args) { @Override public String getDescription() { - return "Parameters: [dop] [orders-input] [lineitem-input] [output]"; + return "Parameters: [parallelism] [orders-input] [lineitem-input] [output]"; } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java index 925ed5c4224a4..c00d231740c39 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java @@ -81,11 +81,11 @@ */ @SuppressWarnings({"serial", "deprecation"}) public class TPCHQuery9 implements Program, ProgramDescription { - public final String ARGUMENTS = "dop partInputPath partSuppInputPath ordersInputPath lineItemInputPath supplierInputPath nationInputPath outputPath"; + public final String ARGUMENTS = "parallelism partInputPath partSuppInputPath ordersInputPath lineItemInputPath supplierInputPath nationInputPath outputPath"; private static Logger LOG = LoggerFactory.getLogger(TPCHQuery9.class); - private int degreeOfParallelism = 1; + private int parallelism = 1; private String partInputPath, partSuppInputPath, ordersInputPath, lineItemInputPath, supplierInputPath, nationInputPath; @@ -100,7 +100,7 @@ public Plan getPlan(String... args) throws IllegalArgumentException { { LOG.warn("number of arguments do not match!"); - this.degreeOfParallelism = 1; + this.parallelism = 1; this.partInputPath = ""; this.partSuppInputPath = ""; this.ordersInputPath = ""; @@ -110,7 +110,7 @@ public Plan getPlan(String... args) throws IllegalArgumentException { this.outputPath = ""; }else { - this.degreeOfParallelism = Integer.parseInt(args[0]); + this.parallelism = Integer.parseInt(args[0]); this.partInputPath = args[1]; this.partSuppInputPath = args[2]; this.ordersInputPath = args[3]; @@ -240,7 +240,7 @@ public Plan getPlan(String... args) throws IllegalArgumentException { result.setInput(sumAmountAggregate); Plan p = new Plan(result, "TPC-H query 9"); - p.setDefaultParallelism(this.degreeOfParallelism); + p.setDefaultParallelism(this.parallelism); return p; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java index 415fde99bb477..a681f64c71c81 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java @@ -147,7 +147,7 @@ public Plan getPlan(final String... args) { */ // create DataSourceContract for Orders input FileDataSource orders = new FileDataSource(new CsvInputFormat(), ordersPath, "Orders"); - orders.setDegreeOfParallelism(numSubtasks); + orders.setParallelism(numSubtasks); CsvInputFormat.configureRecordFormat(orders) .recordDelimiter('\n') .fieldDelimiter('|') @@ -160,7 +160,7 @@ public Plan getPlan(final String... args) { */ // create DataSourceContract for Customer input FileDataSource customers = new FileDataSource(new CsvInputFormat(), customerPath, "Customers"); - customers.setDegreeOfParallelism(numSubtasks); + customers.setParallelism(numSubtasks); CsvInputFormat.configureRecordFormat(customers) .recordDelimiter('\n') .fieldDelimiter('|') @@ -171,17 +171,17 @@ public Plan getPlan(final String... args) { JoinOperator joinCO = JoinOperator.builder(new JoinCO(), IntValue.class, 0, 0) .name("JoinCO") .build(); - joinCO.setDegreeOfParallelism(numSubtasks); + joinCO.setParallelism(numSubtasks); // create ReduceOperator for aggregating the result ReduceOperator aggCO = ReduceOperator.builder(new AggCO(), StringValue.class, 1) .name("AggCo") .build(); - aggCO.setDegreeOfParallelism(numSubtasks); + aggCO.setParallelism(numSubtasks); // create DataSinkContract for writing the result FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, "Output"); - result.setDegreeOfParallelism(numSubtasks); + result.setParallelism(numSubtasks); CsvOutputFormat.configureRecordFormat(result) .recordDelimiter('\n') .fieldDelimiter('|') diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java index 45889ead7565b..377c7eec79f91 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java @@ -318,7 +318,7 @@ public Plan getPlan(String... args) { // Create DataSinkContract for writing the result of the OLAP query FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, antiJoinVisits, "Result"); - result.setDegreeOfParallelism(numSubTasks); + result.setParallelism(numSubTasks); CsvOutputFormat.configureRecordFormat(result) .recordDelimiter('\n') .fieldDelimiter('|') diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java index 860e6b9c8343e..4563c074bb39f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java @@ -59,12 +59,12 @@ public Plan getPlan(String... args) throws IllegalArgumentException { // This task will read the input data and generate the key/value pairs final FileDataSource source = new FileDataSource(new TeraInputFormat(), input, "Data Source"); - source.setDegreeOfParallelism(numSubTasks); + source.setParallelism(numSubTasks); // This task writes the sorted data back to disk final FileDataSink sink = new FileDataSink(new TeraOutputFormat(), output, "Data Sink"); - sink.setDegreeOfParallelism(numSubTasks); + sink.setParallelism(numSubTasks); sink.setGlobalOrder(new Ordering(0, TeraKey.class, Order.ASCENDING), new TeraDistribution()); sink.setInput(source); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java index 6f5e6985510c4..ec498ea1fb837 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java @@ -174,7 +174,7 @@ public void testTaskManagerProcessFailure() { final Throwable[] errorRef = new Throwable[1]; ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort); - env.setDegreeOfParallelism(PARALLELISM); + env.setParallelism(PARALLELISM); env.setNumberOfExecutionRetries(1); env.getConfig().setExecutionMode(executionMode); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java index 48afce1c6c53b..42b1c15e528e3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java @@ -79,7 +79,7 @@ public void testFailedRunThenSuccessfulRun() { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(4); + env.setParallelism(4); env.setNumberOfExecutionRetries(0); env.generateSequence(1, 10) @@ -108,7 +108,7 @@ public Long reduce(Long value1, Long value2) { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(4); + env.setParallelism(4); env.setNumberOfExecutionRetries(0); env.generateSequence(1, 10) @@ -154,7 +154,7 @@ public void testRestart() { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(4); + env.setParallelism(4); env.setNumberOfExecutionRetries(1); env.generateSequence(1, 10) @@ -198,7 +198,7 @@ public void testRestartMultipleTimes() { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(4); + env.setParallelism(4); env.setNumberOfExecutionRetries(3); env.generateSequence(1, 10) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java index eb042347b89c8..aa92925b10088 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java @@ -85,7 +85,7 @@ public void testRestartWithFailingTaskManager() { final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getJobManagerRPCPort()); - env.setDegreeOfParallelism(PARALLELISM); + env.setParallelism(PARALLELISM); env.setNumberOfExecutionRetries(1); env.generateSequence(1, 10) diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java index 007742b0de32b..1cdb736d96217 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java @@ -31,7 +31,7 @@ public class ConsumePipelinedAndBlockingResultITCase extends JavaProgramTestBase @Override protected void testProgram() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(1); + env.setParallelism(1); DataSet> pipelinedSource = env.fromElements(new Tuple1(1l)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java index 784e82452486b..6d5e00eb7954f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java @@ -58,7 +58,7 @@ public Plan getPlan(String... args) { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setDegreeOfParallelism(4); + env.setParallelism(4); // get input data DataSet points = env.readCsvFile(pointsPath) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala index 33e080709d33e..b6f045b8d5764 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala @@ -106,7 +106,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * check correctness of groupReduce on tuples with key field selector and group sorting */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(1).sortGroup(2, Order.ASCENDING).reduceGroup { in => @@ -301,7 +301,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas val env = ExecutionEnvironment.getExecutionEnvironment // important because it determines how often the combiner is called - env.setDegreeOfParallelism(2) + env.setParallelism(2) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine) @@ -341,7 +341,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * check correctness of groupReduce with descending group sort */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(1).sortGroup(2, Order.DESCENDING).reduceGroup { in => @@ -386,7 +386,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * sorting */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env).map { t => MutableTuple3(t._1, t._2, t._3) } @@ -478,7 +478,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * check correctness of groupReduce with descending group sort */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(1) .sortGroup("_3", Order.DESCENDING) @@ -500,7 +500,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * Test int-based definition on group sort, for (full) nested Tuple */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) val reduceDs = ds.groupBy("_2").sortGroup(0, Order.DESCENDING) .reduceGroup(new NestedTupleReducer) @@ -515,7 +515,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * Test int-based definition on group sort, for (partial) nested Tuple ASC */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) val reduceDs = ds.groupBy("_2") .sortGroup("_1._1", Order.ASCENDING) @@ -532,7 +532,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * Test string-based definition on group sort, for (partial) nested Tuple DESC */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) val reduceDs = ds.groupBy("_2") .sortGroup("_1._1", Order.DESCENDING) @@ -549,7 +549,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * Test string-based definition on group sort, for two grouping keys */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env) val reduceDs = ds.groupBy("_2") .sortGroup("_1._1", Order.DESCENDING) @@ -566,7 +566,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * Test string-based definition on group sort, for two grouping keys with Pojos */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env) val reduceDs = ds.groupBy("hadoopFan") .sortGroup("theTuple._1", Order.DESCENDING) @@ -601,7 +601,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * check correctness of sorted groupReduce on tuples with keyselector sorting */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(_._2).sortGroup(_._3, Order.DESCENDING).reduceGroup { in => @@ -658,7 +658,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * check correctness of sorted groupReduce with combine on tuples with keyselector sorting */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.get3TupleDataSet(env) val reduceDs = ds.groupBy(_._2).sortGroup(_._3, Order.DESCENDING) @@ -683,7 +683,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * check correctness of sorted groupReduceon with Tuple2 keyselector sorting */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.get5TupleDataSet(env) val reduceDs = ds.groupBy(_._1).sortGroup(t => (t._5, t._3), Order.DESCENDING).reduceGroup{ @@ -722,7 +722,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas * Test grouping with pojo containing multiple pojos (was a bug) */ val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(1) + env.setParallelism(1) val ds = CollectionDataSets.getPojoWithMultiplePojos(env) val reduceDs = ds.groupBy("p2.a2") .reduceGroup { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala index c5ced67d8fa0e..9bf3ccea754cf 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala @@ -289,7 +289,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) .where("nestedPojo.longNumber", "number", "str") .equalTo("_7", "_1", "_2") joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.setDegreeOfParallelism(1) + env.setParallelism(1) env.execute() expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One," + "10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two," + @@ -307,7 +307,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) val joinDs = ds1.join(ds2).where("nestedPojo.longNumber", "number", "nestedTupleWithCustom._1").equalTo("_7", "_1", "_3") joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.setDegreeOfParallelism(1) + env.setParallelism(1) env.execute() expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One," + "10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two," + @@ -328,7 +328,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) "nestedTupleWithCustom._2.myLong") .equalTo("_3", "_4", "_5") joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.setDegreeOfParallelism(1) + env.setParallelism(1) env.execute() expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One," + "10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two," + @@ -345,7 +345,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) val ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env) val joinDs = ds1.join(ds2).where(0).equalTo("_1._1", "_1._2") joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.setDegreeOfParallelism(1) + env.setParallelism(1) env.execute() expected = "((1,1),one),((1,1),one)\n" + "((2,2),two),((2,2),two)\n" + "((3,3),three),((3,3)," + "three)\n" @@ -362,7 +362,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) val ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env) val joinDs = ds1.join(ds2).where("_1._1").equalTo("_1._1") joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.setDegreeOfParallelism(1) + env.setParallelism(1) env.execute() expected = "((1,1),one),((1,1),one)\n" + "((2,2),two),((2,2),two)\n" + "((3,3),three),((3,3)," + "three)\n" @@ -378,7 +378,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) val ds2 = CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env) val joinDs = ds1.join(ds2).where("*").equalTo("*") joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.setDegreeOfParallelism(1) + env.setParallelism(1) env.execute() expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" + "2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" + diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/PartitionITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/PartitionITCase.scala index 8e00075761018..98bb44661a57c 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/PartitionITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/PartitionITCase.scala @@ -103,20 +103,20 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase( countsInPartition.writeAsText(resultPath, WriteMode.OVERWRITE) env.execute() - val numPerPartition : Int = 2220 / env.getDegreeOfParallelism / 10 + val numPerPartition : Int = 2220 / env.getParallelism / 10 expected = "" - for (i <- 0 until env.getDegreeOfParallelism) { + for (i <- 0 until env.getParallelism) { expected += "(" + i + "," + numPerPartition + ")\n" } } @Test - def testMapPartitionAfterRepartitionHasCorrectDOP(): Unit = { + def testMapPartitionAfterRepartitionHasCorrectParallelism(): Unit = { // Verify that mapPartition operation after repartition picks up correct - // DOP + // parallelism val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) - env.setDegreeOfParallelism(1) + env.setParallelism(1) val unique = ds.partitionByHash(1) .setParallelism(4) @@ -129,12 +129,12 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase( } @Test - def testMapAfterRepartitionHasCorrectDOP(): Unit = { + def testMapAfterRepartitionHasCorrectParallelism(): Unit = { // Verify that map operation after repartition picks up correct - // DOP + // parallelism val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) - env.setDegreeOfParallelism(1) + env.setParallelism(1) val count = ds.partitionByHash(0).setParallelism(4).map( new RichMapFunction[(Int, Long, String), Tuple1[Int]] { @@ -157,12 +157,12 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase( } @Test - def testFilterAfterRepartitionHasCorrectDOP(): Unit = { + def testFilterAfterRepartitionHasCorrectParallelism(): Unit = { // Verify that filter operation after repartition picks up correct - // DOP + // parallelism val env = ExecutionEnvironment.getExecutionEnvironment val ds = CollectionDataSets.get3TupleDataSet(env) - env.setDegreeOfParallelism(1) + env.setParallelism(1) val count = ds.partitionByHash(0).setParallelism(4).filter( new RichFilterFunction[(Int, Long, String)] { @@ -188,7 +188,7 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase( @Test def testPartitionNestedPojo(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(3) + env.setParallelism(3) val ds = CollectionDataSets.getDuplicatePojoDataSet(env) val uniqLongs = ds .partitionByHash("nestedPojo.longNumber") diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala index ae3a20c26a8aa..425cff657e939 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala @@ -31,9 +31,9 @@ class AggregateTranslationTest { @Test def translateAggregate(): Unit = { try { - val DOP = 8 + val parallelism = 8 - val env = ExecutionEnvironment.createLocalEnvironment(DOP) + val env = ExecutionEnvironment.createLocalEnvironment(parallelism) val initialData = env.fromElements((3.141592, "foobar", 77L)) @@ -46,7 +46,7 @@ class AggregateTranslationTest { assertEquals(1, reducer.getKeyColumns(0).length) assertEquals(0, reducer.getKeyColumns(0)(0)) - assertEquals(-1, reducer.getDegreeOfParallelism) + assertEquals(-1, reducer.getParallelism) assertTrue(reducer.isCombinable) assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]]) } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala index 969f9709d5bde..11de8d53e239e 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala @@ -37,7 +37,7 @@ class CustomPartitioningTest extends CompilerTestBase { val parallelism = 4 val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(parallelism) + env.setParallelism(parallelism) val data = env.fromElements( (0,0) ).rebalance() @@ -80,7 +80,7 @@ class CustomPartitioningTest extends CompilerTestBase { val parallelism = 4 val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(parallelism) + env.setParallelism(parallelism) val data = env.fromElements( (0,0) ).rebalance() try { @@ -106,7 +106,7 @@ class CustomPartitioningTest extends CompilerTestBase { val parallelism = 4 val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(parallelism) + env.setParallelism(parallelism) val data = env.fromElements(new Pojo()).rebalance() @@ -150,7 +150,7 @@ class CustomPartitioningTest extends CompilerTestBase { val parallelism = 4 val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(parallelism) + env.setParallelism(parallelism) val data = env.fromElements(new Pojo()).rebalance() @@ -177,7 +177,7 @@ class CustomPartitioningTest extends CompilerTestBase { val parallelism = 4 val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(parallelism) + env.setParallelism(parallelism) val data = env.fromElements(new Pojo()).rebalance() diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala index 0246500db1e9b..6aa4d75859b43 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala @@ -46,11 +46,11 @@ class DeltaIterationTranslationTest { val AGGREGATOR_NAME = "AggregatorName" val ITERATION_KEYS = Array(2) val NUM_ITERATIONS = 13 - val DEFAULT_DOP = 133 - val ITERATION_DOP = 77 + val DEFAULT_PARALLELISM = 133 + val ITERATION_PARALLELISM = 77 val env = ExecutionEnvironment.getExecutionEnvironment - env.setDegreeOfParallelism(DEFAULT_DOP) + env.setParallelism(DEFAULT_PARALLELISM) val initialSolutionSet = env.fromElements((3.44, 5L, "abc")) val initialWorkSet = env.fromElements((1.23, "abc")) @@ -64,7 +64,7 @@ class DeltaIterationTranslationTest { (joined, joined.map(new NextWorksetMapper).name(BEFORE_NEXT_WORKSET_MAP)) } result.name(ITERATION_NAME) - .setParallelism(ITERATION_DOP) + .setParallelism(ITERATION_PARALLELISM) .registerAggregator(AGGREGATOR_NAME, new LongSumAggregator) result.print() @@ -72,7 +72,7 @@ class DeltaIterationTranslationTest { val p: Plan = env.createProgramPlan(JOB_NAME) assertEquals(JOB_NAME, p.getJobName) - assertEquals(DEFAULT_DOP, p.getDefaultParallelism) + assertEquals(DEFAULT_PARALLELISM, p.getDefaultParallelism) var sink1: GenericDataSinkBase[_] = null var sink2: GenericDataSinkBase[_] = null val sinks = p.getDataSinks.iterator @@ -85,7 +85,7 @@ class DeltaIterationTranslationTest { assertEquals(iteration, sink2.getInput) assertEquals(NUM_ITERATIONS, iteration.getMaximumNumberOfIterations) assertArrayEquals(ITERATION_KEYS, iteration.getSolutionSetKeyFields) - assertEquals(ITERATION_DOP, iteration.getDegreeOfParallelism) + assertEquals(ITERATION_PARALLELISM, iteration.getParallelism) assertEquals(ITERATION_NAME, iteration.getName) val nextWorksetMapper: MapOperatorBase[_, _, _] = diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala index 79a584d4ef3cf..e97fc218153d1 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala @@ -33,8 +33,8 @@ class ReduceTranslationTest { @Test def translateNonGroupedReduce(): Unit = { try { - val DOP = 8 - val env = ExecutionEnvironment.createLocalEnvironment(DOP) + val parallelism = 8 + val env = ExecutionEnvironment.createLocalEnvironment(parallelism) val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1) @@ -50,7 +50,7 @@ class ReduceTranslationTest { assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getInputType) assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getOutputType) assertTrue(reducer.getKeyColumns(0) == null || reducer.getKeyColumns(0).length == 0) - assertTrue(reducer.getDegreeOfParallelism == 1 || reducer.getDegreeOfParallelism == -1) + assertTrue(reducer.getParallelism == 1 || reducer.getParallelism == -1) assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]]) } catch { @@ -65,8 +65,8 @@ class ReduceTranslationTest { @Test def translateGroupedReduceNoMapper(): Unit = { try { - val DOP: Int = 8 - val env = ExecutionEnvironment.createLocalEnvironment(DOP) + val parallelism: Int = 8 + val env = ExecutionEnvironment.createLocalEnvironment(parallelism) val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1) @@ -78,7 +78,7 @@ class ReduceTranslationTest { val reducer: ReduceOperatorBase[_, _] = sink.getInput.asInstanceOf[ReduceOperatorBase[_, _]] assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getInputType) assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getOutputType) - assertTrue(reducer.getDegreeOfParallelism == DOP || reducer.getDegreeOfParallelism == -1) + assertTrue(reducer.getParallelism == parallelism || reducer.getParallelism == -1) assertArrayEquals(Array[Int](2), reducer.getKeyColumns(0)) assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]]) } @@ -94,8 +94,8 @@ class ReduceTranslationTest { @Test def translateGroupedReduceWithKeyExtractor(): Unit = { try { - val DOP: Int = 8 - val env = ExecutionEnvironment.createLocalEnvironment(DOP) + val parallelism: Int = 8 + val env = ExecutionEnvironment.createLocalEnvironment(parallelism) val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1) @@ -109,9 +109,9 @@ class ReduceTranslationTest { .asInstanceOf[PlanUnwrappingReduceOperator[_, _]] val keyExtractor: MapOperatorBase[_, _, _] = reducer.getInput .asInstanceOf[MapOperatorBase[_, _, _]] - assertEquals(1, keyExtractor.getDegreeOfParallelism) - assertEquals(4, reducer.getDegreeOfParallelism) - assertEquals(4, keyProjector.getDegreeOfParallelism) + assertEquals(1, keyExtractor.getParallelism) + assertEquals(4, reducer.getParallelism) + assertEquals(4, keyProjector.getParallelism) val keyValueInfo = new TupleTypeInfo( BasicTypeInfo.STRING_TYPE_INFO, createTypeInformation[(Double, String, Long)]) diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala index ce17d72c1784e..9c3311bbdf560 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala @@ -170,7 +170,7 @@ object ApplicationMaster { if(slots != -1){ output.println(s"${ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS}: $slots") output.println( - s"${ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY}: ${slots*taskManagerCount}") + s"${ConfigConstants.DEFAULT_PARALLELISM_KEY}: ${slots*taskManagerCount}") } output.println(s"${ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION}: " + From 658bb1166aa907677e06cf011e5a0fdaf58ab15f Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 18 Mar 2015 10:44:44 +0100 Subject: [PATCH 3/4] [FLINK-1679] deprecate old parallelism config entry old config parameter can still be used OLD parallelization.degree.default NEW parallelism.default --- .../org/apache/flink/configuration/ConfigConstants.java | 9 ++++++++- .../main/java/org/apache/flink/optimizer/Optimizer.java | 9 +++++++-- .../api/environment/StreamContextEnvironment.java | 7 ++++++- .../streaming/api/environment/StreamPlanEnvironment.java | 7 ++++++- 4 files changed, 27 insertions(+), 5 deletions(-) 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 b472d8afdb2f6..09f55fd1a9e12 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 @@ -34,7 +34,14 @@ public final class ConfigConstants { * The config parameter defining the default parallelism for jobs. */ public static final String DEFAULT_PARALLELISM_KEY = "parallelism.default"; - + + /** + * The deprecated config parameter defining the default parallelism for jobs. + */ + @Deprecated + public static final String DEFAULT_PARALLELISM_KEY_OLD = "parallelization.degree.default"; + + /** * Config parameter for the number of re-tries for failed tasks. Setting this * value to 0 effectively disables fault tolerance. diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java index 90421b7dec1a4..c80cfc271ef21 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java @@ -348,10 +348,15 @@ public Optimizer(DataStatistics stats, CostEstimator estimator) { this.costEstimator = estimator; // determine the default parallelism + // check for old key string first, then for new one this.defaultParallelism = GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD, ConfigConstants.DEFAULT_PARALLELISM); - + // now check for new one which overwrites old values + this.defaultParallelism = GlobalConfiguration.getInteger( + ConfigConstants.DEFAULT_PARALLELISM_KEY, + this.defaultParallelism); + if (defaultParallelism < 1) { LOG.warn("Config value " + defaultParallelism + " for option " + ConfigConstants.DEFAULT_PARALLELISM + " is invalid. Ignoring and using a value of 1."); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java index 7ae78f172334a..f7dd0bff6901a 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java @@ -38,9 +38,14 @@ protected StreamContextEnvironment(Client client, List jars, int paralleli if (parallelism > 0) { setParallelism(parallelism); } else { + // first check for old parallelism config key setParallelism(GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD, ConfigConstants.DEFAULT_PARALLELISM)); + // then for new + setParallelism(GlobalConfiguration.getInteger( + ConfigConstants.DEFAULT_PARALLELISM_KEY, + getParallelism())); } } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java index 2cf5cc2fecc46..592fa1a2c82b5 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java @@ -36,9 +36,14 @@ protected StreamPlanEnvironment(ExecutionEnvironment env) { if (parallelism > 0) { setParallelism(parallelism); } else { + // first check for old parallelism config key setParallelism(GlobalConfiguration.getInteger( - ConfigConstants.DEFAULT_PARALLELISM_KEY, + ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD, ConfigConstants.DEFAULT_PARALLELISM)); + // then for new + setParallelism(GlobalConfiguration.getInteger( + ConfigConstants.DEFAULT_PARALLELISM_KEY, + getParallelism())); } } From 412ac54df0fde12666afbc1414df5fd919ba1607 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 18 Mar 2015 10:44:45 +0100 Subject: [PATCH 4/4] [FLINK-1679] extend faq and programming guide to clarify parallelism --- docs/config.md | 16 ++++++++---- docs/faq.md | 21 +++++++++++++++ docs/programming_guide.md | 54 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 86 insertions(+), 5 deletions(-) diff --git a/docs/config.md b/docs/config.md index 09a01d9d17a8f..3524a605a63dd 100644 --- a/docs/config.md +++ b/docs/config.md @@ -80,11 +80,17 @@ This value is typically proportional to the number of physical CPU cores that the TaskManager's machine has (e.g., equal to the number of cores, or half the number of cores). [More about task slots](config.html#configuring-taskmanager-processing-slots). -- `parallelism.default`: The default parallelism to use for -programs that have no parallelism specified. (DEFAULT: 1). For -setups that have no concurrent jobs running, setting this value to -NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all -available execution resources for the program's execution. +- `parallelism.default`: The default parallelism to use for programs that have +no parallelism specified. (DEFAULT: 1). For setups that have no concurrent jobs +running, setting this value to NumTaskManagers * NumSlotsPerTaskManager will +cause the system to use all available execution resources for the program's +execution. **Note**: The default parallelism can be overwriten for an entire +job by calling `setParallelism(int parallelism)` on the `ExecutionEnvironment` +or by passing `-p ` to the Flink Command-line frontend. It can be +overwritten for single transformations by calling `setParallelism(int +parallelism)` on an operator. See the [programming +guide](programming_guide.html#parallel-execution) for more information about the +parallelism. - `fs.hdfs.hadoopconf`: The absolute path to the Hadoop File System's (HDFS) configuration directory (OPTIONAL VALUE). diff --git a/docs/faq.md b/docs/faq.md index 2c3fe53b3962d..5c31bc66dc6be 100644 --- a/docs/faq.md +++ b/docs/faq.md @@ -85,6 +85,27 @@ tracking the behavior of the parallel execution. They allow you to gather information inside the program's operations and show them after the program execution. +### What is the parallelism? How do I set it? + +In Flink programs, the parallelism determines how operations are split into +individual tasks which are assigned to task slots. Each node in a cluster has at +least one task slot. The total number of task slots is the number of all task slots +on all machines. If the parallelism is set to `N`, Flink tries to divide an +operation into `N` parallel tasks which can be computed concurrently using the +available task slots. The number of task slots should be equal to the +parallelism to ensure that all tasks can be computed in a task slot concurrently. + +**Note**: Not all operations can be divided into multiple tasks. For example, a +`GroupReduce` operation without a grouping has to be performed with a +parallelism of 1 because the entire group needs to be present at exactly one +node to perform the reduce operation. Flink will determine whether the +parallelism has to be 1 and set it accordingly. + +The parallelism can be set in numerous ways to ensure a fine-grained control +over the execution of a Flink program. See +[Configuration](config.html#common-options) for detailed instructions on how to +set the parallelism. + ## Errors ### Why am I getting a "NonSerializableException" ? diff --git a/docs/programming_guide.md b/docs/programming_guide.md index 1750a12c71ae3..4e15b912bba09 100644 --- a/docs/programming_guide.md +++ b/docs/programming_guide.md @@ -2789,6 +2789,60 @@ env.execute("Word Count Example")
  • +### Client Level + +The parallelism can be set at the Client when submitting jobs to Flink. The +Client can either be a Java or a Scala program. One example of such a Client is +Flink's Command-line Interface (CLI). + +For the CLI client, the parallelism parameter can be specified with `-p`. For +exampple: + + ./bin/flink run -p 10 ../examples/*WordCount-java*.jar + + +In a Java/Scala program, the parallelism is set as follows: + +
    +
    +{% highlight java %} + +try { + PackagedProgram program = new PackagedProgram(file, args); + InetSocketAddress jobManagerAddress = RemoteExecutor.getInetFromHostport("localhost:6123"); + Configuration config = new Configuration(); + + Client client = new Client(jobManagerAddress, config, program.getUserCodeClassLoader()); + + // set the parallelism to 10 here + client.run(program, 10, true); + +} catch (ProgramInvocationException e) { + e.printStackTrace(); +} + +{% endhighlight %} +
    +
    +{% highlight scala %} +try { + PackagedProgram program = new PackagedProgram(file, args) + InetSocketAddress jobManagerAddress = RemoteExecutor.getInetFromHostport("localhost:6123") + Configuration config = new Configuration() + + Client client = new Client(jobManagerAddress, new Configuration(), program.getUserCodeClassLoader()) + + // set the parallelism to 10 here + client.run(program, 10, true) + +} catch { + case e: Exception => e.printStackTrace +} +{% endhighlight %} +
    +
    + + ### System Level A system-wide default parallelism for all execution environments can be defined by setting the