You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/03/17 10:16:21 UTC
[1/8] flink git commit: [optimizer] Remove obsolete and unused fields
from OptimizerNode
Repository: flink
Updated Branches:
refs/heads/master 1950d0568 -> 1c50d87c1
[optimizer] Remove obsolete and unused fields from OptimizerNode
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3b579adf
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3b579adf
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3b579adf
Branch: refs/heads/master
Commit: 3b579adf08c1370fe393c0dcf6887f08369c36d8
Parents: 1950d05
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Mar 10 15:14:09 2015 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Mar 17 10:10:07 2015 +0100
----------------------------------------------------------------------
.../flink/compiler/dag/OptimizerNode.java | 31 +-------------------
1 file changed, 1 insertion(+), 30 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/3b579adf/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
index 55df592..4e4ef28 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
@@ -88,8 +88,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
// --------------------------------- General Parameters ---------------------------------------
private int degreeOfParallelism = -1; // the number of parallel instances of this node
-
- private int subtasksPerInstance = -1; // the number of parallel instance that will run on the same machine
private long minimalMemoryPerSubTask = -1;
@@ -100,8 +98,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
protected boolean onDynamicPath;
protected List<PlanNode> cachedPlans; // cache candidates, because the may be accessed repeatedly
-
- protected int[][] remappedKeys;
// ------------------------------------------------------------------------
// Constructor / Setup
@@ -115,17 +111,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
public OptimizerNode(Operator<?> op) {
this.pactContract = op;
readStubAnnotations();
-
- if (this.pactContract instanceof AbstractUdfOperator) {
- final AbstractUdfOperator<?, ?> pact = (AbstractUdfOperator<?, ?>) this.pactContract;
- this.remappedKeys = new int[pact.getNumberOfInputs()][];
- for (int i = 0; i < this.remappedKeys.length; i++) {
- final int[] keys = pact.getKeyColumns(i);
- int[] rk = new int[keys.length];
- System.arraycopy(keys, 0, rk, 0, keys.length);
- this.remappedKeys[i] = rk;
- }
- }
}
protected OptimizerNode(OptimizerNode toCopy) {
@@ -133,8 +118,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
this.intProps = toCopy.intProps;
- this.remappedKeys = toCopy.remappedKeys;
-
this.openBranches = toCopy.openBranches;
this.closedBranchingNodes = toCopy.closedBranchingNodes;
@@ -142,7 +125,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
this.estimatedNumRecords = toCopy.estimatedNumRecords;
this.degreeOfParallelism = toCopy.degreeOfParallelism;
- this.subtasksPerInstance = toCopy.subtasksPerInstance;
this.minimalMemoryPerSubTask = toCopy.minimalMemoryPerSubTask;
this.id = toCopy.id;
@@ -259,7 +241,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
public abstract List<PlanNode> getAlternativePlans(CostEstimator estimator);
/**
- * This method implements the visit of a depth-first graph traversing visitor. Implementors must first
+ * This method implements the visit of a depth-first graph traversing visitor. Implementers must first
* call the <code>preVisit()</code> method, then hand the visitor to their children, and finally call
* the <code>postVisit()</code> method.
*
@@ -666,13 +648,6 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
// ------------------------------------------------------------------------
// Access of stub annotations
// ------------------------------------------------------------------------
-
- /**
- * An optional method where nodes can describe which fields will be unique in their output.
- */
- public List<FieldSet> createUniqueFieldsForNode() {
- return null;
- }
/**
* Gets the FieldSets which are unique in the output of the node.
@@ -1157,8 +1132,4 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
return bld.toString();
}
-
- public int[] getRemappedKeys(int input) {
- return this.remappedKeys[input];
- }
}
[8/8] flink git commit: [FLINK-1671] [jobmanager] Rename JobManager's
ExecutionMode to JobManagerMode
Posted by se...@apache.org.
[FLINK-1671] [jobmanager] Rename JobManager's ExecutionMode to JobManagerMode
This is done to avoid name conflicts with the overloaded type name ExecutionMode.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/361c90a8
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/361c90a8
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/361c90a8
Branch: refs/heads/master
Commit: 361c90a847d8da3f5bcae4efb4fdc3b83a9cdcf1
Parents: 3b579ad
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Mar 10 18:22:20 2015 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Mar 17 10:10:08 2015 +0100
----------------------------------------------------------------------
.../runtime/jobmanager/JobManagerMode.java | 37 ++++++++++++++++++++
.../flink/runtime/jobmanager/JobManager.scala | 31 ++++++++++------
.../jobmanager/JobManagerCLIConfiguration.scala | 16 ++++-----
.../JobManagerProcessReapingTest.java | 2 +-
.../jobmanager/JobManagerStartupTest.java | 4 +--
5 files changed, 68 insertions(+), 22 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/361c90a8/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerMode.java
new file mode 100644
index 0000000..201176b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerMode.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.jobmanager;
+
+/**
+ * The startup mode for the JobManager.
+ */
+public enum JobManagerMode {
+
+ /**
+ * Causes the JobManager to operate in single user mode and
+ * start a local embedded TaskManager.
+ */
+ LOCAL,
+
+ /**
+ * Starts the JobManager in the regular mode where it waits for external TaskManagers
+ * to connect.
+ */
+ CLUSTER
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/361c90a8/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index c350680..98dd7eb 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -629,8 +629,6 @@ class JobManager(val configuration: Configuration,
* look up the JobManager actor reference.
*/
object JobManager {
-
- import ExecutionMode._
val LOG = LoggerFactory.getLogger(classOf[JobManager])
@@ -656,7 +654,7 @@ object JobManager {
// parsing the command line arguments
val (configuration: Configuration,
- executionMode: ExecutionMode,
+ executionMode: JobManagerMode,
listeningHost: String, listeningPort: Int) =
try {
parseArgs(args)
@@ -717,7 +715,7 @@ object JobManager {
* @param listeningPort The port where the JobManager should listen for messages.
*/
def runJobManager(configuration: Configuration,
- executionMode: ExecutionMode,
+ executionMode: JobManagerMode,
listeningAddress: String,
listeningPort: Int) : Unit = {
@@ -761,7 +759,7 @@ object JobManager {
"JobManager_Process_Reaper")
// bring up a local task manager, if needed
- if (executionMode.equals(LOCAL)) {
+ if (executionMode == JobManagerMode.LOCAL) {
LOG.info("Starting embedded TaskManager for JobManager's LOCAL execution mode")
TaskManager.startTaskManagerActor(configuration, jobManagerSystem, listeningAddress,
@@ -798,18 +796,29 @@ object JobManager {
* @param args command line arguments
* @return Quadruple of configuration, execution mode and an optional listening address
*/
- def parseArgs(args: Array[String]): (Configuration, ExecutionMode, String, Int) = {
+ def parseArgs(args: Array[String]): (Configuration, JobManagerMode, String, Int) = {
val parser = new scopt.OptionParser[JobManagerCLIConfiguration]("JobManager") {
head("Flink JobManager")
+
opt[String]("configDir") action { (arg, c) => c.copy(configDir = arg) } text {
"The configuration directory." }
+
opt[String]("executionMode") action { (arg, c) =>
- if (arg.equalsIgnoreCase("local")){
- c.copy(executionMode = LOCAL)
- } else if (arg.equalsIgnoreCase("cluster")) {
- c.copy(executionMode = CLUSTER)
- } else {
+ val argLower = arg.toLowerCase()
+ var result: JobManagerCLIConfiguration = null
+
+ for (mode <- JobManagerMode.values() if result == null) {
+ val modeName = mode.name().toLowerCase()
+
+ if (modeName.equals(argLower)) {
+ result = c.copy(executionMode = mode)
+ }
+ }
+
+ if (result == null) {
throw new Exception("Unknown execution mode: " + arg)
+ } else {
+ result
}
} text {
"The execution mode of the JobManager (CLUSTER / LOCAL)"
http://git-wip-us.apache.org/repos/asf/flink/blob/361c90a8/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala
index 905dbe3..4cd02c5 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala
@@ -18,11 +18,11 @@
package org.apache.flink.runtime.jobmanager
-object ExecutionMode extends Enumeration{
- type ExecutionMode = Value
- val LOCAL = Value
- val CLUSTER = Value
-}
-
-case class JobManagerCLIConfiguration(configDir: String = null,
- executionMode: ExecutionMode.ExecutionMode = null) {}
+/**
+ * Holder for command line parameters of the JobManager.
+ *
+ * @param configDir The directory to load the configuration from.
+ * @param executionMode Mode for the JobManager.
+ */
+case class JobManagerCLIConfiguration(configDir: String = null,
+ executionMode: JobManagerMode = null) {}
http://git-wip-us.apache.org/repos/asf/flink/blob/361c90a8/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
index e1f9b4a..eee945a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessReapingTest.java
@@ -183,7 +183,7 @@ public class JobManagerProcessReapingTest {
Configuration config = new Configuration();
config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1);
- JobManager.runJobManager(config, ExecutionMode.CLUSTER(), "localhost", port);
+ JobManager.runJobManager(config, JobManagerMode.CLUSTER, "localhost", port);
System.exit(0);
}
catch (Throwable t) {
http://git-wip-us.apache.org/repos/asf/flink/blob/361c90a8/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
index 4884b19..070e376 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java
@@ -80,7 +80,7 @@ public class JobManagerStartupTest {
}
try {
- JobManager.runJobManager(new Configuration(), ExecutionMode.CLUSTER(), "localhost", portNum);
+ JobManager.runJobManager(new Configuration(), JobManagerMode.CLUSTER, "localhost", portNum);
fail("this should throw an exception");
}
catch (Exception e) {
@@ -117,7 +117,7 @@ public class JobManagerStartupTest {
failConfig.setString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, nonExistDirectory);
try {
- JobManager.runJobManager(failConfig, ExecutionMode.CLUSTER(), "localhost", portNum);
+ JobManager.runJobManager(failConfig, JobManagerMode.CLUSTER, "localhost", portNum);
fail("this should fail with an exception");
}
catch (Exception e) {
[6/8] flink git commit: [FLINK-1671] [optimizer] Add data exchange
mode to optimizer classes
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/GlobalProperties.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/GlobalProperties.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/GlobalProperties.java
index 31e13ae..7670277 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/GlobalProperties.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/GlobalProperties.java
@@ -21,6 +21,7 @@ package org.apache.flink.compiler.dataproperties;
import java.util.HashSet;
import java.util.Set;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.common.operators.Order;
import org.apache.flink.api.common.operators.Ordering;
@@ -30,16 +31,15 @@ import org.apache.flink.api.common.operators.util.FieldSet;
import org.apache.flink.compiler.CompilerException;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.util.Utils;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class represents global properties of the data at a certain point in the plan.
- * Global properties are properties that describe data across different partitions.
- * <p>
- * Currently, the properties are the following: A partitioning type (ANY, HASH, RANGE), and EITHER an ordering (for range partitioning)
- * or an FieldSet with the hash partitioning columns.
+ * Global properties are properties that describe data across different partitions, such as
+ * whether the data is hash partitioned, range partitioned, replicated, etc.
*/
public class GlobalProperties implements Cloneable {
@@ -67,9 +67,9 @@ public class GlobalProperties implements Cloneable {
// --------------------------------------------------------------------------------------------
/**
- * Sets the partitioning property for the global properties.
+ * Sets this global properties to represent a hash partitioning.
*
- * @param partitionedFields
+ * @param partitionedFields The key fields on which the data is hash partitioned.
*/
public void setHashPartitioned(FieldList partitionedFields) {
if (partitionedFields == null) {
@@ -355,30 +355,64 @@ public class GlobalProperties implements Cloneable {
}
- public void parameterizeChannel(Channel channel, boolean globalDopChange) {
+ public void parameterizeChannel(Channel channel, boolean globalDopChange,
+ ExecutionMode exchangeMode, boolean breakPipeline) {
+
+ ShipStrategyType shipType;
+ FieldList partitionKeys;
+ boolean[] sortDirection;
+ Partitioner<?> partitioner;
+
switch (this.partitioning) {
case RANDOM_PARTITIONED:
- channel.setShipStrategy(globalDopChange ? ShipStrategyType.PARTITION_RANDOM : ShipStrategyType.FORWARD);
+ shipType = globalDopChange ? ShipStrategyType.PARTITION_RANDOM : ShipStrategyType.FORWARD;
+ partitionKeys = null;
+ sortDirection = null;
+ partitioner = null;
break;
+
case FULL_REPLICATION:
- channel.setShipStrategy(ShipStrategyType.BROADCAST);
+ shipType = ShipStrategyType.BROADCAST;
+ partitionKeys = null;
+ sortDirection = null;
+ partitioner = null;
break;
+
case ANY_PARTITIONING:
case HASH_PARTITIONED:
- channel.setShipStrategy(ShipStrategyType.PARTITION_HASH, Utils.createOrderedFromSet(this.partitioningFields));
+ shipType = ShipStrategyType.PARTITION_HASH;
+ partitionKeys = Utils.createOrderedFromSet(this.partitioningFields);
+ sortDirection = null;
+ partitioner = null;
break;
+
case RANGE_PARTITIONED:
- channel.setShipStrategy(ShipStrategyType.PARTITION_RANGE, this.ordering.getInvolvedIndexes(), this.ordering.getFieldSortDirections());
+ shipType = ShipStrategyType.PARTITION_RANGE;
+ partitionKeys = this.ordering.getInvolvedIndexes();
+ sortDirection = this.ordering.getFieldSortDirections();
+ partitioner = null;
break;
+
case FORCED_REBALANCED:
- channel.setShipStrategy(ShipStrategyType.PARTITION_RANDOM);
+ shipType = ShipStrategyType.PARTITION_RANDOM;
+ partitionKeys = null;
+ sortDirection = null;
+ partitioner = null;
break;
+
case CUSTOM_PARTITIONING:
- channel.setShipStrategy(ShipStrategyType.PARTITION_CUSTOM, this.partitioningFields, this.customPartitioner);
+ shipType = ShipStrategyType.PARTITION_CUSTOM;
+ partitionKeys = this.partitioningFields;
+ sortDirection = null;
+ partitioner = this.customPartitioner;
break;
+
default:
throw new CompilerException("Unsupported partitioning strategy");
}
+
+ DataExchangeMode exMode = DataExchangeMode.select(exchangeMode, shipType, breakPipeline);
+ channel.setShipStrategy(shipType, partitionKeys, sortDirection, partitioner, exMode);
}
// ------------------------------------------------------------------------
@@ -438,7 +472,7 @@ public class GlobalProperties implements Cloneable {
// --------------------------------------------------------------------------------------------
- public static final GlobalProperties combine(GlobalProperties gp1, GlobalProperties gp2) {
+ public static GlobalProperties combine(GlobalProperties gp1, GlobalProperties gp2) {
if (gp1.isFullyReplicated()) {
if (gp2.isFullyReplicated()) {
return new GlobalProperties();
@@ -448,7 +482,7 @@ public class GlobalProperties implements Cloneable {
} else if (gp2.isFullyReplicated()) {
return gp1;
} else if (gp1.ordering != null) {
- return gp1;
+ return gp1;
} else if (gp2.ordering != null) {
return gp2;
} else if (gp1.partitioningFields != null) {
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/RequestedGlobalProperties.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/RequestedGlobalProperties.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/RequestedGlobalProperties.java
index f4334ff..10c1248 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/RequestedGlobalProperties.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dataproperties/RequestedGlobalProperties.java
@@ -18,6 +18,7 @@
package org.apache.flink.compiler.dataproperties;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.distributions.DataDistribution;
import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.common.operators.Ordering;
@@ -27,14 +28,20 @@ import org.apache.flink.api.common.operators.util.FieldSet;
import org.apache.flink.compiler.CompilerException;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.util.Utils;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
/**
- * This class represents global properties of the data that an operator is interested in, because it needs those
- * properties for its contract.
- * <p>
- * Currently, the properties are the following: A partitioning type (ANY, HASH, RANGE), and EITHER an ordering (for range partitioning)
- * or an FieldSet with the hash partitioning columns.
+ * This class represents the global properties of the data that are requested by an operator.
+ * Operators request the global properties they need for correct execution. This list is an example of global
+ * properties requested by certain operators:
+ * <ul>
+ * <li>"groupBy/reduce" will request the data to be partitioned in some way after the key fields.</li>
+ * <li>"map" will request the data to be in an arbitrary distribution - it has no prerequisites</li>
+ * <li>"join" will request certain properties for each input. This class represents the properties
+ * on an input alone. The properties may be partitioning on the key fields, or a combination of
+ * replication on one input and anything-but-replication on the other input.</li>
+ * </ul>
*/
public final class RequestedGlobalProperties implements Cloneable {
@@ -60,11 +67,13 @@ public final class RequestedGlobalProperties implements Cloneable {
// --------------------------------------------------------------------------------------------
/**
- * Sets the partitioning property for the global properties.
- * If the partitionFields are provided as {@link FieldSet} also subsets are valid,
- * if provided as {@link FieldList} partitioning fields must exactly match incl. order.
+ * Sets these properties to request a hash partitioning on the given fields.
*
- * @param partitionedFields
+ * If the fields are provided as {@link FieldSet}, then any permutation of the fields is a
+ * valid partitioning, including subsets. If the fields are given as a {@link FieldList},
+ * then only an exact partitioning on the fields matches this requested partitioning.
+ *
+ * @param partitionedFields The key fields for the partitioning.
*/
public void setHashPartitioned(FieldSet partitionedFields) {
if (partitionedFields == null) {
@@ -91,11 +100,14 @@ public final class RequestedGlobalProperties implements Cloneable {
}
/**
- * Sets the partitioning property for the global properties.
- * If the partitionFields are provided as {@link FieldSet} also subsets are valid,
- * if provided as {@link FieldList} partitioning fields must exactly match incl. order.
+ * Sets these properties to request some partitioning on the given fields. This will allow
+ * both hash partitioning and range partitioning to match.
+ *
+ * If the fields are provided as {@link FieldSet}, then any permutation of the fields is a
+ * valid partitioning, including subsets. If the fields are given as a {@link FieldList},
+ * then only an exact partitioning on the fields matches this requested partitioning.
*
- * @param partitionedFields
+ * @param partitionedFields The key fields for the partitioning.
*/
public void setAnyPartitioning(FieldSet partitionedFields) {
if (partitionedFields == null) {
@@ -131,11 +143,13 @@ public final class RequestedGlobalProperties implements Cloneable {
}
/**
- * Sets the partitioning property for the global properties.
- * If the partitionFields are provided as {@link FieldSet} also subsets are valid,
- * if provided as {@link FieldList} partitioning fields must exactly match incl. order.
+ * Sets these properties to request a custom partitioning with the given {@link Partitioner} instance.
*
- * @param partitionedFields
+ * If the fields are provided as {@link FieldSet}, then any permutation of the fields is a
+ * valid partitioning, including subsets. If the fields are given as a {@link FieldList},
+ * then only an exact partitioning on the fields matches this requested partitioning.
+ *
+ * @param partitionedFields The key fields for the partitioning.
*/
public void setCustomPartitioned(FieldSet partitionedFields, Partitioner<?> partitioner) {
if (partitionedFields == null || partitioner == null) {
@@ -322,63 +336,102 @@ public final class RequestedGlobalProperties implements Cloneable {
}
/**
- * Parameterizes the ship strategy fields of a channel such that the channel produces the desired global properties.
+ * Parametrizes the ship strategy fields of a channel such that the channel produces
+ * the desired global properties.
*
- * @param channel The channel to parameterize.
- * @param globalDopChange
+ * @param channel The channel to parametrize.
+ * @param globalDopChange Flag indicating whether the degree of parallelism changes
+ * between sender and receiver.
+ * @param exchangeMode The mode of data exchange (pipelined, always batch,
+ * batch only on shuffle, ...)
+ * @param breakPipeline Indicates whether this data exchange should break
+ * pipelines (unless pipelines are forced).
*/
- public void parameterizeChannel(Channel channel, boolean globalDopChange) {
+ public void parameterizeChannel(Channel channel, boolean globalDopChange,
+ ExecutionMode exchangeMode, boolean breakPipeline) {
// safety check. Fully replicated input must be preserved.
- if(channel.getSource().getGlobalProperties().isFullyReplicated() &&
+ if (channel.getSource().getGlobalProperties().isFullyReplicated() &&
!(this.partitioning == PartitioningProperty.FULL_REPLICATION ||
- this.partitioning == PartitioningProperty.ANY_DISTRIBUTION)) {
- throw new CompilerException("Fully replicated input must be preserved and may not be converted into another global property.");
+ this.partitioning == PartitioningProperty.ANY_DISTRIBUTION))
+ {
+ throw new CompilerException("Fully replicated input must be preserved " +
+ "and may not be converted into another global property.");
}
// if we request nothing, then we need no special strategy. forward, if the number of instances remains
// the same, randomly repartition otherwise
if (isTrivial() || this.partitioning == PartitioningProperty.ANY_DISTRIBUTION) {
- channel.setShipStrategy(globalDopChange ? ShipStrategyType.PARTITION_RANDOM : ShipStrategyType.FORWARD);
+ ShipStrategyType shipStrategy = globalDopChange ? ShipStrategyType.PARTITION_RANDOM :
+ ShipStrategyType.FORWARD;
+
+ DataExchangeMode em = DataExchangeMode.select(exchangeMode, shipStrategy, breakPipeline);
+ channel.setShipStrategy(shipStrategy, em);
return;
}
final GlobalProperties inGlobals = channel.getSource().getGlobalProperties();
// if we have no global parallelism change, check if we have already compatible global properties
if (!globalDopChange && isMetBy(inGlobals)) {
- channel.setShipStrategy(ShipStrategyType.FORWARD);
+ DataExchangeMode em = DataExchangeMode.select(exchangeMode, ShipStrategyType.FORWARD, breakPipeline);
+ channel.setShipStrategy(ShipStrategyType.FORWARD, em);
return;
}
// if we fall through the conditions until here, we need to re-establish
+ ShipStrategyType shipType;
+ FieldList partitionKeys;
+ boolean[] sortDirection;
+ Partitioner<?> partitioner;
+
switch (this.partitioning) {
case FULL_REPLICATION:
- channel.setShipStrategy(ShipStrategyType.BROADCAST);
+ shipType = ShipStrategyType.BROADCAST;
+ partitionKeys = null;
+ sortDirection = null;
+ partitioner = null;
break;
-
+
case ANY_PARTITIONING:
case HASH_PARTITIONED:
- channel.setShipStrategy(ShipStrategyType.PARTITION_HASH, Utils.createOrderedFromSet(this.partitioningFields));
+ shipType = ShipStrategyType.PARTITION_HASH;
+ partitionKeys = Utils.createOrderedFromSet(this.partitioningFields);
+ sortDirection = null;
+ partitioner = null;
break;
case RANGE_PARTITIONED:
- channel.setShipStrategy(ShipStrategyType.PARTITION_RANGE, this.ordering.getInvolvedIndexes(), this.ordering.getFieldSortDirections());
- if(this.dataDistribution != null) {
+ shipType = ShipStrategyType.PARTITION_RANGE;
+ partitionKeys = this.ordering.getInvolvedIndexes();
+ sortDirection = this.ordering.getFieldSortDirections();
+ partitioner = null;
+
+ if (this.dataDistribution != null) {
channel.setDataDistribution(this.dataDistribution);
}
break;
-
+
case FORCED_REBALANCED:
- channel.setShipStrategy(ShipStrategyType.PARTITION_FORCED_REBALANCE);
+ shipType = ShipStrategyType.PARTITION_FORCED_REBALANCE;
+ partitionKeys = null;
+ sortDirection = null;
+ partitioner = null;
break;
-
+
case CUSTOM_PARTITIONING:
- channel.setShipStrategy(ShipStrategyType.PARTITION_CUSTOM, Utils.createOrderedFromSet(this.partitioningFields), this.customPartitioner);
+ shipType = ShipStrategyType.PARTITION_CUSTOM;
+ partitionKeys = Utils.createOrderedFromSet(this.partitioningFields);
+ sortDirection = null;
+ partitioner = this.customPartitioner;
break;
-
+
default:
- throw new CompilerException("Invalid partitioning to create through a data exchange: " + this.partitioning.name());
+ throw new CompilerException("Invalid partitioning to create through a data exchange: "
+ + this.partitioning.name());
}
+
+ DataExchangeMode exMode = DataExchangeMode.select(exchangeMode, shipType, breakPipeline);
+ channel.setShipStrategy(shipType, partitionKeys, sortDirection, partitioner, exMode);
}
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllGroupWithPartialPreGroupProperties.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllGroupWithPartialPreGroupProperties.java b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllGroupWithPartialPreGroupProperties.java
index 54885a7..ec38b47 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllGroupWithPartialPreGroupProperties.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllGroupWithPartialPreGroupProperties.java
@@ -31,6 +31,7 @@ import org.apache.flink.compiler.dataproperties.RequestedGlobalProperties;
import org.apache.flink.compiler.dataproperties.RequestedLocalProperties;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.plan.SingleInputPlanNode;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
@@ -45,24 +46,29 @@ public final class AllGroupWithPartialPreGroupProperties extends OperatorDescrip
public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
if (in.getShipStrategy() == ShipStrategyType.FORWARD) {
// locally connected, directly instantiate
- return new SingleInputPlanNode(node, "GroupReduce ("+node.getPactContract().getName()+")", in, DriverStrategy.ALL_GROUP_REDUCE);
+ return new SingleInputPlanNode(node, "GroupReduce ("+node.getPactContract().getName()+")",
+ in, DriverStrategy.ALL_GROUP_REDUCE);
} else {
// non forward case.plug in a combiner
Channel toCombiner = new Channel(in.getSource());
- toCombiner.setShipStrategy(ShipStrategyType.FORWARD);
+ toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
// create an input node for combine with same DOP as input node
GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode();
combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
- SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_GROUP_COMBINE);
+ SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode,
+ "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_GROUP_COMBINE);
combiner.setCosts(new Costs(0, 0));
combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties());
Channel toReducer = new Channel(combiner);
- toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(), in.getShipStrategySortOrder());
+ toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(),
+ in.getShipStrategySortOrder(), in.getDataExchangeMode());
+
toReducer.setLocalStrategy(in.getLocalStrategy(), in.getLocalStrategyKeys(), in.getLocalStrategySortOrder());
- return new SingleInputPlanNode(node, "GroupReduce ("+node.getPactContract().getName()+")", toReducer, DriverStrategy.ALL_GROUP_REDUCE);
+ return new SingleInputPlanNode(node, "GroupReduce ("+node.getPactContract().getName()+")",
+ toReducer, DriverStrategy.ALL_GROUP_REDUCE);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllReduceProperties.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllReduceProperties.java b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllReduceProperties.java
index 2bf757e..17fa318 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllReduceProperties.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/AllReduceProperties.java
@@ -30,11 +30,11 @@ import org.apache.flink.compiler.dataproperties.RequestedGlobalProperties;
import org.apache.flink.compiler.dataproperties.RequestedLocalProperties;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.plan.SingleInputPlanNode;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-public final class AllReduceProperties extends OperatorDescriptorSingle
-{
+public final class AllReduceProperties extends OperatorDescriptorSingle {
@Override
public DriverStrategy getStrategy() {
@@ -45,24 +45,30 @@ public final class AllReduceProperties extends OperatorDescriptorSingle
public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
if (in.getShipStrategy() == ShipStrategyType.FORWARD) {
// locally connected, directly instantiate
- return new SingleInputPlanNode(node, "Reduce ("+node.getPactContract().getName()+")", in, DriverStrategy.ALL_REDUCE);
+ return new SingleInputPlanNode(node, "Reduce ("+node.getPactContract().getName()+")",
+ in, DriverStrategy.ALL_REDUCE);
} else {
// non forward case.plug in a combiner
Channel toCombiner = new Channel(in.getSource());
- toCombiner.setShipStrategy(ShipStrategyType.FORWARD);
+ toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
// create an input node for combine with same DOP as input node
ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode();
combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
- SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_REDUCE);
+ SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode,
+ "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.ALL_REDUCE);
combiner.setCosts(new Costs(0, 0));
combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties());
Channel toReducer = new Channel(combiner);
- toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(), in.getShipStrategySortOrder());
- toReducer.setLocalStrategy(in.getLocalStrategy(), in.getLocalStrategyKeys(), in.getLocalStrategySortOrder());
- return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", toReducer, DriverStrategy.ALL_REDUCE);
+ toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(),
+ in.getShipStrategySortOrder(), in.getDataExchangeMode());
+ toReducer.setLocalStrategy(in.getLocalStrategy(), in.getLocalStrategyKeys(),
+ in.getLocalStrategySortOrder());
+
+ return new SingleInputPlanNode(node, "Reduce ("+node.getPactContract().getName()+")",
+ toReducer, DriverStrategy.ALL_REDUCE);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/operators/GroupReduceWithCombineProperties.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/GroupReduceWithCombineProperties.java b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/GroupReduceWithCombineProperties.java
index fd263e6..7180845 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/GroupReduceWithCombineProperties.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/GroupReduceWithCombineProperties.java
@@ -35,6 +35,7 @@ import org.apache.flink.compiler.dataproperties.RequestedGlobalProperties;
import org.apache.flink.compiler.dataproperties.RequestedLocalProperties;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.plan.SingleInputPlanNode;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.runtime.operators.util.LocalStrategy;
@@ -94,13 +95,16 @@ public final class GroupReduceWithCombineProperties extends OperatorDescriptorSi
if (!in.getLocalStrategyKeys().isValidUnorderedPrefix(this.keys)) {
throw new RuntimeException("Bug: Inconsistent sort for group strategy.");
}
- in.setLocalStrategy(LocalStrategy.COMBININGSORT, in.getLocalStrategyKeys(), in.getLocalStrategySortOrder());
+ in.setLocalStrategy(LocalStrategy.COMBININGSORT, in.getLocalStrategyKeys(),
+ in.getLocalStrategySortOrder());
}
- return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", in, DriverStrategy.SORTED_GROUP_REDUCE, this.keyList);
+ return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", in,
+ DriverStrategy.SORTED_GROUP_REDUCE, this.keyList);
} else {
// non forward case. all local properties are killed anyways, so we can safely plug in a combiner
Channel toCombiner = new Channel(in.getSource());
- toCombiner.setShipStrategy(ShipStrategyType.FORWARD);
+ toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
+
// create an input node for combine with same DOP as input node
GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode();
combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
@@ -115,9 +119,13 @@ public final class GroupReduceWithCombineProperties extends OperatorDescriptorSi
combiner.setDriverKeyInfo(this.keyList, 1);
Channel toReducer = new Channel(combiner);
- toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(), in.getShipStrategySortOrder());
- toReducer.setLocalStrategy(LocalStrategy.COMBININGSORT, in.getLocalStrategyKeys(), in.getLocalStrategySortOrder());
- return new SingleInputPlanNode(node, "Reduce ("+node.getPactContract().getName()+")", toReducer, DriverStrategy.SORTED_GROUP_REDUCE, this.keyList);
+ toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(),
+ in.getShipStrategySortOrder(), in.getDataExchangeMode());
+ toReducer.setLocalStrategy(LocalStrategy.COMBININGSORT, in.getLocalStrategyKeys(),
+ in.getLocalStrategySortOrder());
+
+ return new SingleInputPlanNode(node, "Reduce ("+node.getPactContract().getName()+")",
+ toReducer, DriverStrategy.SORTED_GROUP_REDUCE, this.keyList);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/operators/ReduceProperties.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/ReduceProperties.java b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/ReduceProperties.java
index 000079d..3a054ff 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/operators/ReduceProperties.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/operators/ReduceProperties.java
@@ -33,6 +33,7 @@ import org.apache.flink.compiler.dataproperties.RequestedGlobalProperties;
import org.apache.flink.compiler.dataproperties.RequestedLocalProperties;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.plan.SingleInputPlanNode;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.runtime.operators.util.LocalStrategy;
@@ -60,25 +61,32 @@ public final class ReduceProperties extends OperatorDescriptorSingle {
if (in.getShipStrategy() == ShipStrategyType.FORWARD ||
(node.getBroadcastConnections() != null && !node.getBroadcastConnections().isEmpty()))
{
- return new SingleInputPlanNode(node, "Reduce ("+node.getPactContract().getName()+")", in, DriverStrategy.SORTED_REDUCE, this.keyList);
+ return new SingleInputPlanNode(node, "Reduce ("+node.getPactContract().getName()+")", in,
+ DriverStrategy.SORTED_REDUCE, this.keyList);
}
else {
// non forward case. all local properties are killed anyways, so we can safely plug in a combiner
Channel toCombiner = new Channel(in.getSource());
- toCombiner.setShipStrategy(ShipStrategyType.FORWARD);
+ toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
// create an input node for combine with same DOP as input node
ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode();
combinerNode.setDegreeOfParallelism(in.getSource().getDegreeOfParallelism());
- SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine ("+node.getPactContract().getName()+")", toCombiner, DriverStrategy.SORTED_PARTIAL_REDUCE, this.keyList);
+ SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode,
+ "Combine ("+node.getPactContract().getName()+")", toCombiner,
+ DriverStrategy.SORTED_PARTIAL_REDUCE, this.keyList);
+
combiner.setCosts(new Costs(0, 0));
combiner.initProperties(toCombiner.getGlobalProperties(), toCombiner.getLocalProperties());
Channel toReducer = new Channel(combiner);
- toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(), in.getShipStrategySortOrder());
+ toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(),
+ in.getShipStrategySortOrder(), in.getDataExchangeMode());
toReducer.setLocalStrategy(LocalStrategy.SORT, in.getLocalStrategyKeys(), in.getLocalStrategySortOrder());
- return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", toReducer, DriverStrategy.SORTED_REDUCE, this.keyList);
+
+ return new SingleInputPlanNode(node, "Reduce("+node.getPactContract().getName()+")", toReducer,
+ DriverStrategy.SORTED_REDUCE, this.keyList);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
index e159481..3903c84 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/Channel.java
@@ -32,6 +32,7 @@ import org.apache.flink.compiler.dataproperties.RequestedGlobalProperties;
import org.apache.flink.compiler.dataproperties.RequestedLocalProperties;
import org.apache.flink.compiler.plandump.DumpableConnection;
import org.apache.flink.compiler.util.Utils;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.runtime.operators.util.LocalStrategy;
@@ -43,8 +44,10 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
private PlanNode source;
private PlanNode target;
-
+
private ShipStrategyType shipStrategy = ShipStrategyType.NONE;
+
+ private DataExchangeMode dataExchangeMode;
private LocalStrategy localStrategy = LocalStrategy.NONE;
@@ -78,8 +81,6 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
private double relativeTempMemory;
- private double relativeMemoryGlobalStrategy;
-
private double relativeMemoryLocalStrategy;
private int replicationFactor = 1;
@@ -125,33 +126,46 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
public PlanNode getTarget() {
return this.target;
}
-
- public void setShipStrategy(ShipStrategyType strategy) {
- setShipStrategy(strategy, null, null, null);
+
+ public void setShipStrategy(ShipStrategyType strategy, DataExchangeMode dataExchangeMode) {
+ setShipStrategy(strategy, null, null, null, dataExchangeMode);
}
- public void setShipStrategy(ShipStrategyType strategy, FieldList keys) {
- setShipStrategy(strategy, keys, null, null);
+ public void setShipStrategy(ShipStrategyType strategy, FieldList keys, DataExchangeMode dataExchangeMode) {
+ setShipStrategy(strategy, keys, null, null, dataExchangeMode);
}
- public void setShipStrategy(ShipStrategyType strategy, FieldList keys, boolean[] sortDirection) {
- setShipStrategy(strategy, keys, sortDirection, null);
+ public void setShipStrategy(ShipStrategyType strategy, FieldList keys,
+ boolean[] sortDirection, DataExchangeMode dataExchangeMode) {
+ setShipStrategy(strategy, keys, sortDirection, null, dataExchangeMode);
}
- public void setShipStrategy(ShipStrategyType strategy, FieldList keys, Partitioner<?> partitioner) {
- setShipStrategy(strategy, keys, null, partitioner);
+ public void setShipStrategy(ShipStrategyType strategy, FieldList keys,
+ Partitioner<?> partitioner, DataExchangeMode dataExchangeMode) {
+ setShipStrategy(strategy, keys, null, partitioner, dataExchangeMode);
}
- public void setShipStrategy(ShipStrategyType strategy, FieldList keys, boolean[] sortDirection, Partitioner<?> partitioner) {
+ public void setShipStrategy(ShipStrategyType strategy, FieldList keys,
+ boolean[] sortDirection, Partitioner<?> partitioner,
+ DataExchangeMode dataExchangeMode) {
this.shipStrategy = strategy;
this.shipKeys = keys;
this.shipSortOrder = sortDirection;
this.partitioner = partitioner;
-
+ this.dataExchangeMode = dataExchangeMode;
this.globalProps = null; // reset the global properties
}
-
-
+
+ /**
+ * Gets the data exchange mode (batch / streaming) to use for the data
+ * exchange of this channel.
+ *
+ * @return The data exchange mode of this channel.
+ */
+ public DataExchangeMode getDataExchangeMode() {
+ return dataExchangeMode;
+ }
+
public ShipStrategyType getShipStrategy() {
return this.shipStrategy;
}
@@ -168,10 +182,6 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
setLocalStrategy(strategy, null, null);
}
- public void setLocalStrategy(LocalStrategy strategy, FieldList keys) {
- setLocalStrategy(strategy, keys, null);
- }
-
public void setLocalStrategy(LocalStrategy strategy, FieldList keys, boolean[] sortDirection) {
this.localStrategy = strategy;
this.localKeys = keys;
@@ -307,14 +317,6 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
this.localStrategyComparator = localStrategyComparator;
}
- public double getRelativeMemoryGlobalStrategy() {
- return relativeMemoryGlobalStrategy;
- }
-
- public void setRelativeMemoryGlobalStrategy(double relativeMemoryGlobalStrategy) {
- this.relativeMemoryGlobalStrategy = relativeMemoryGlobalStrategy;
- }
-
public double getRelativeMemoryLocalStrategy() {
return relativeMemoryLocalStrategy;
}
@@ -477,8 +479,6 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
/**
* Utility method used while swapping binary union nodes for n-ary union nodes.
- *
- * @param newUnionNode
*/
public void swapUnionNodes(PlanNode newUnionNode) {
if (!(this.source instanceof BinaryUnionPlanNode)) {
@@ -493,16 +493,17 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
public int getMaxDepth() {
return this.source.getOptimizerNode().getMaxDepth() + 1;
}
- // --------------------------------------------------------------------------------------------
-
+ // --------------------------------------------------------------------------------------------
+ @Override
public String toString() {
return "Channel (" + this.source + (this.target == null ? ')' : ") -> (" + this.target + ')') +
'[' + this.shipStrategy + "] [" + this.localStrategy + "] " +
(this.tempMode == null || this.tempMode == TempMode.NONE ? "{NO-TEMP}" : this.tempMode);
}
-
+
+ @Override
public Channel clone() {
try {
return (Channel) super.clone();
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/plan/OptimizedPlan.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/OptimizedPlan.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/OptimizedPlan.java
index 00eb287..51d65e1 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plan/OptimizedPlan.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plan/OptimizedPlan.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.compiler.plan;
import java.util.Collection;
@@ -26,64 +25,46 @@ import org.apache.flink.util.Visitable;
import org.apache.flink.util.Visitor;
/**
- * The optimizer representation of a plan. The optimizer creates this from the user defined PACT job plan.
- * It works on this representation during its optimization. Finally, this plan is translated to a schedule
- * for the nephele runtime system.
+ * The execution plan generated by the Optimizer. It contains {@link PlanNode}s
+ * and {@link Channel}s that describe exactly how the program should be executed.
+ * It defines all ship strategies (local pipe, shuffle, broadcast, rebalance), all
+ * operator strategies (sorting-merge join, hash join, sorted grouping, ...),
+ * and the data exchange modes (batched, pipelined).
*/
public class OptimizedPlan implements FlinkPlan, Visitable<PlanNode> {
- /**
- * The data sources in the plan.
- */
+ /** The data sources in the plan. */
private final Collection<SourcePlanNode> dataSources;
- /**
- * The data sinks in the plan.
- */
+ /** The data sinks in the plan. */
private final Collection<SinkPlanNode> dataSinks;
- /**
- * All nodes in the optimizer plan.
- */
+ /** All nodes in the optimizer plan. */
private final Collection<PlanNode> allNodes;
- /**
- * The original pact plan.
- */
- private final Plan pactPlan;
+ /** The original program. */
+ private final Plan originalProgram;
- /**
- * Name of the PACT job
- */
+ /** Name of the job */
private final String jobName;
/**
- * The name of the instance type that is to be used.
- */
- private String instanceTypeName;
-
-
- /**
* Creates a new instance of this optimizer plan container. The plan is given and fully
* described by the data sources, sinks and the collection of all nodes.
*
- * @param sources
- * The nodes describing the data sources.
- * @param sinks
- * The nodes describing the data sinks.
- * @param allNodes
- * A collection containing all nodes in the plan.
- * @param jobName
- * The name of the PACT job
+ * @param sources The data sources.
+ * @param sinks The data sinks.
+ * @param allNodes A collection containing all nodes in the plan.
+ * @param jobName The name of the program
*/
public OptimizedPlan(Collection<SourcePlanNode> sources, Collection<SinkPlanNode> sinks,
- Collection<PlanNode> allNodes, String jobName, Plan pactPlan)
+ Collection<PlanNode> allNodes, String jobName, Plan programPlan)
{
this.dataSources = sources;
this.dataSinks = sinks;
this.allNodes = allNodes;
this.jobName = jobName;
- this.pactPlan = pactPlan;
+ this.originalProgram = programPlan;
}
/**
@@ -114,46 +95,27 @@ public class OptimizedPlan implements FlinkPlan, Visitable<PlanNode> {
}
/**
- * Returns the name of the optimized PACT job.
+ * Returns the name of the program.
*
- * @return The name of the optimized PACT job.
+ * @return The name of the program.
*/
public String getJobName() {
return this.jobName;
}
/**
- * Gets the original pact plan from which this optimized plan was created.
+ * Gets the original program plan from which this optimized plan was created.
*
- * @return The original pact plan.
+ * @return The original program plan.
*/
public Plan getOriginalPactPlan() {
- return this.pactPlan;
- }
-
- /**
- * Gets the name of the instance type that should be used for this PACT job.
- *
- * @return The instance-type name.
- */
- public String getInstanceTypeName() {
- return instanceTypeName;
- }
-
- /**
- * Sets the name of the instance type that should be used for this PACT job.
- *
- * @param instanceTypeName
- * The name of the instance type.
- */
- public void setInstanceTypeName(String instanceTypeName) {
- this.instanceTypeName = instanceTypeName;
+ return this.originalProgram;
}
// ------------------------------------------------------------------------
/**
- * Takes the given visitor and applies it top down to all nodes, starting at the sinks.
+ * Applies the given visitor top down to all nodes, starting at the sinks.
*
* @param visitor
* The visitor to apply to the nodes in this plan.
@@ -165,5 +127,4 @@ public class OptimizedPlan implements FlinkPlan, Visitable<PlanNode> {
node.accept(visitor);
}
}
-
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/AdditionalOperatorsTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/AdditionalOperatorsTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/AdditionalOperatorsTest.java
index cf32126..07fc972 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/AdditionalOperatorsTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/AdditionalOperatorsTest.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.compiler;
import static org.junit.Assert.assertEquals;
@@ -28,7 +27,6 @@ import org.apache.flink.api.java.record.operators.CrossWithLargeOperator;
import org.apache.flink.api.java.record.operators.CrossWithSmallOperator;
import org.apache.flink.api.java.record.operators.FileDataSink;
import org.apache.flink.api.java.record.operators.FileDataSource;
-import org.apache.flink.compiler.CompilerException;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.plan.DualInputPlanNode;
import org.apache.flink.compiler.plan.OptimizedPlan;
@@ -38,7 +36,6 @@ import org.apache.flink.compiler.util.DummyOutputFormat;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.junit.Test;
-
/**
* Tests that validate optimizer choices when using operators that are requesting certain specific execution
* strategies.
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java b/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java
index c6a9b55..beea0b9 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/CompilerTestBase.java
@@ -20,19 +20,16 @@ package org.apache.flink.compiler;
import java.util.ArrayList;
import java.util.HashMap;
-import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
import org.apache.flink.api.common.operators.GenericDataSourceBase;
import org.apache.flink.api.common.operators.Operator;
-import org.apache.flink.api.java.record.operators.BulkIteration;
-import org.apache.flink.api.java.record.operators.DeltaIteration;
+import org.apache.flink.api.common.operators.base.BulkIterationBase;
import org.apache.flink.compiler.costs.DefaultCostEstimator;
import org.apache.flink.compiler.plan.OptimizedPlan;
import org.apache.flink.compiler.plan.PlanNode;
@@ -42,9 +39,10 @@ import org.apache.flink.util.Visitor;
import org.junit.Before;
/**
- *
+ * Base class for Optimizer tests. Offers utility methods to trigger optimization
+ * of a program and to fetch the nodes in an optimizer plan that correspond
+ * the the node in the program plan.
*/
-@SuppressWarnings("deprecation")
public abstract class CompilerTestBase implements java.io.Serializable {
private static final long serialVersionUID = 1L;
@@ -55,8 +53,6 @@ public abstract class CompilerTestBase implements java.io.Serializable {
protected static final int DEFAULT_PARALLELISM = 8;
- protected static final String DEFAULT_PARALLELISM_STRING = String.valueOf(DEFAULT_PARALLELISM);
-
private static final String CACHE_KEY = "cachekey";
// ------------------------------------------------------------------------
@@ -102,16 +98,11 @@ public abstract class CompilerTestBase implements java.io.Serializable {
}
// ------------------------------------------------------------------------
- public static OperatorResolver getContractResolver(Plan plan) {
- return new OperatorResolver(plan);
- }
public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) {
return new OptimizerPlanNodeResolver(plan);
}
- // ------------------------------------------------------------------------
-
public static final class OptimizerPlanNodeResolver {
private final Map<String, ArrayList<PlanNode>> map;
@@ -205,97 +196,6 @@ public abstract class CompilerTestBase implements java.io.Serializable {
}
}
}
-
- // ------------------------------------------------------------------------
-
- public static final class OperatorResolver implements Visitor<Operator<?>> {
-
- private final Map<String, List<Operator<?>>> map;
- private Set<Operator<?>> seen;
-
- OperatorResolver(Plan p) {
- this.map = new HashMap<String, List<Operator<?>>>();
- this.seen = new HashSet<Operator<?>>();
-
- p.accept(this);
- this.seen = null;
- }
-
-
- @SuppressWarnings("unchecked")
- public <T extends Operator<?>> T getNode(String name) {
- List<Operator<?>> nodes = this.map.get(name);
- if (nodes == null || nodes.isEmpty()) {
- throw new RuntimeException("No nodes found with the given name.");
- } else if (nodes.size() != 1) {
- throw new RuntimeException("Multiple nodes found with the given name.");
- } else {
- return (T) nodes.get(0);
- }
- }
-
- @SuppressWarnings("unchecked")
- public <T extends Operator<?>> T getNode(String name, Class<? extends Function> stubClass) {
- List<Operator<?>> nodes = this.map.get(name);
- if (nodes == null || nodes.isEmpty()) {
- throw new RuntimeException("No node found with the given name and stub class.");
- } else {
- Operator<?> found = null;
- for (Operator<?> node : nodes) {
- if (node.getClass() == stubClass) {
- if (found == null) {
- found = node;
- } else {
- throw new RuntimeException("Multiple nodes found with the given name and stub class.");
- }
- }
- }
- if (found == null) {
- throw new RuntimeException("No node found with the given name and stub class.");
- } else {
- return (T) found;
- }
- }
- }
-
- public List<Operator<?>> getNodes(String name) {
- List<Operator<?>> nodes = this.map.get(name);
- if (nodes == null || nodes.isEmpty()) {
- throw new RuntimeException("No node found with the given name.");
- } else {
- return new ArrayList<Operator<?>>(nodes);
- }
- }
-
- @Override
- public boolean preVisit(Operator<?> visitable) {
- if (this.seen.add(visitable)) {
- // add to the map
- final String name = visitable.getName();
- List<Operator<?>> list = this.map.get(name);
- if (list == null) {
- list = new ArrayList<Operator<?>>(2);
- this.map.put(name, list);
- }
- list.add(visitable);
-
- // recurse into bulk iterations
- if (visitable instanceof BulkIteration) {
- ((BulkIteration) visitable).getNextPartialSolution().accept(this);
- } else if (visitable instanceof DeltaIteration) {
- ((DeltaIteration) visitable).getSolutionSetDelta().accept(this);
- ((DeltaIteration) visitable).getNextWorkset().accept(this);
- }
-
- return true;
- } else {
- return false;
- }
- }
-
- @Override
- public void postVisit(Operator<?> visitable) {}
- }
/**
* Collects all DataSources of a plan to add statistics
@@ -311,8 +211,8 @@ public abstract class CompilerTestBase implements java.io.Serializable {
if(visitable instanceof GenericDataSourceBase) {
sources.add((GenericDataSourceBase<?, ?>) visitable);
}
- else if(visitable instanceof BulkIteration) {
- ((BulkIteration) visitable).getNextPartialSolution().accept(this);
+ else if(visitable instanceof BulkIterationBase) {
+ ((BulkIterationBase<?>) visitable).getNextPartialSolution().accept(this);
}
return true;
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java
index 677d9be..b800279 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/FeedbackPropertiesMatchTest.java
@@ -50,6 +50,7 @@ import org.apache.flink.compiler.plan.PlanNode.FeedbackPropertiesMeetRequirement
import org.apache.flink.compiler.testfunctions.DummyFlatJoinFunction;
import org.apache.flink.compiler.testfunctions.IdentityMapper;
import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.runtime.operators.util.LocalStrategy;
@@ -67,12 +68,12 @@ public class FeedbackPropertiesMatchTest {
SourcePlanNode otherTarget = new SourcePlanNode(getSourceNode(), "Source");
Channel toMap1 = new Channel(target);
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map1 = new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP);
Channel toMap2 = new Channel(map1);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map2 = new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP);
@@ -96,12 +97,12 @@ public class FeedbackPropertiesMatchTest {
SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Source");
Channel toMap1 = new Channel(target);
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map1 = new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP);
Channel toMap2 = new Channel(map1);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map2 = new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP);
@@ -674,10 +675,10 @@ public class FeedbackPropertiesMatchTest {
RequestedGlobalProperties reqGp = new RequestedGlobalProperties();
reqGp.setAnyPartitioning(new FieldSet(2, 5));
- toMap1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 5));
+ toMap1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 5), DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
@@ -700,10 +701,10 @@ public class FeedbackPropertiesMatchTest {
RequestedGlobalProperties reqGp = new RequestedGlobalProperties();
reqGp.setAnyPartitioning(new FieldSet(2, 5));
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
- toMap2.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 5));
+ toMap2.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 5), DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
@@ -726,10 +727,10 @@ public class FeedbackPropertiesMatchTest {
RequestedLocalProperties reqLp = new RequestedLocalProperties();
reqLp.setGroupedFields(new FieldList(4, 1));
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false});
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
toMap1.setRequiredGlobalProps(null);
@@ -751,10 +752,10 @@ public class FeedbackPropertiesMatchTest {
RequestedLocalProperties reqLp = new RequestedLocalProperties();
reqLp.setGroupedFields(new FieldList(4, 1));
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false});
@@ -780,10 +781,10 @@ public class FeedbackPropertiesMatchTest {
RequestedLocalProperties reqLp = new RequestedLocalProperties();
reqLp.setGroupedFields(new FieldList(5, 7));
- toMap1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(5, 7));
+ toMap1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(5, 7), DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false});
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
toMap1.setRequiredGlobalProps(reqGp);
@@ -824,13 +825,13 @@ public class FeedbackPropertiesMatchTest {
RequestedLocalProperties reqLp = new RequestedLocalProperties();
reqLp.setGroupedFields(new FieldList(4, 1));
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false});
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
- toMap3.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap3.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap3.setLocalStrategy(LocalStrategy.NONE);
toMap1.setRequiredGlobalProps(null);
@@ -855,13 +856,13 @@ public class FeedbackPropertiesMatchTest {
RequestedGlobalProperties reqGp = new RequestedGlobalProperties();
reqGp.setAnyPartitioning(new FieldSet(2, 3));
- toMap1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(1, 2));
+ toMap1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(1, 2), DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
- toMap3.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap3.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap3.setLocalStrategy(LocalStrategy.NONE);
toMap1.setRequiredGlobalProps(null);
@@ -892,21 +893,21 @@ public class FeedbackPropertiesMatchTest {
SourcePlanNode source2 = new SourcePlanNode(getSourceNode(), "Source 2");
Channel toMap1 = new Channel(source1);
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map1 = new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP);
Channel toMap2 = new Channel(source2);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map2 = new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP);
Channel toJoin1 = new Channel(map1);
Channel toJoin2 = new Channel(map2);
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
- toJoin2.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin2.setLocalStrategy(LocalStrategy.NONE);
DualInputPlanNode join = new DualInputPlanNode(getJoinNode(), "Join", toJoin1, toJoin2, DriverStrategy.HYBRIDHASH_BUILD_FIRST);
@@ -927,12 +928,12 @@ public class FeedbackPropertiesMatchTest {
SourcePlanNode source = new SourcePlanNode(getSourceNode(), "Other Source");
Channel toMap1 = new Channel(target);
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map1 = new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP);
Channel toMap2 = new Channel(source);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map2 = new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP);
@@ -942,13 +943,13 @@ public class FeedbackPropertiesMatchTest {
DualInputPlanNode join = new DualInputPlanNode(getJoinNode(), "Join", toJoin1, toJoin2, DriverStrategy.HYBRIDHASH_BUILD_FIRST);
Channel toAfterJoin = new Channel(join);
- toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD);
+ toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toAfterJoin.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode afterJoin = new SingleInputPlanNode(getMapNode(), "After Join Mapper", toAfterJoin, DriverStrategy.MAP);
// attach some properties to the non-relevant input
{
- toMap2.setShipStrategy(ShipStrategyType.BROADCAST);
+ toMap2.setShipStrategy(ShipStrategyType.BROADCAST, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.SORT, new FieldList(2, 7), new boolean[] {true, true});
RequestedGlobalProperties joinGp = new RequestedGlobalProperties();
@@ -957,7 +958,7 @@ public class FeedbackPropertiesMatchTest {
RequestedLocalProperties joinLp = new RequestedLocalProperties();
joinLp.setOrdering(new Ordering(2, null, Order.ASCENDING).appendOrdering(7, null, Order.ASCENDING));
- toJoin2.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin2.setLocalStrategy(LocalStrategy.NONE);
toJoin2.setRequiredGlobalProps(joinGp);
toJoin2.setRequiredLocalProps(joinLp);
@@ -967,7 +968,7 @@ public class FeedbackPropertiesMatchTest {
// no properties from the partial solution, no required properties
{
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
GlobalProperties gp = new GlobalProperties();
@@ -979,7 +980,7 @@ public class FeedbackPropertiesMatchTest {
// some properties from the partial solution, no required properties
{
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
GlobalProperties gp = new GlobalProperties();
@@ -1006,7 +1007,7 @@ public class FeedbackPropertiesMatchTest {
toJoin1.setRequiredGlobalProps(rgp);
toJoin1.setRequiredLocalProps(rlp);
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
FeedbackPropertiesMeetRequirementsReport report = join.checkPartialSolutionPropertiesMet(target, gp, lp);
@@ -1028,7 +1029,7 @@ public class FeedbackPropertiesMatchTest {
toJoin1.setRequiredGlobalProps(rgp);
toJoin1.setRequiredLocalProps(rlp);
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
FeedbackPropertiesMeetRequirementsReport report = join.checkPartialSolutionPropertiesMet(target, gp, lp);
@@ -1053,7 +1054,7 @@ public class FeedbackPropertiesMatchTest {
toJoin1.setRequiredGlobalProps(null);
toJoin1.setRequiredLocalProps(null);
- toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 1));
+ toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 1), DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false});
FeedbackPropertiesMeetRequirementsReport report = join.checkPartialSolutionPropertiesMet(target, gp, lp);
@@ -1075,13 +1076,13 @@ public class FeedbackPropertiesMatchTest {
toMap1.setRequiredGlobalProps(null);
toMap1.setRequiredLocalProps(null);
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
toJoin1.setRequiredGlobalProps(rgp);
toJoin1.setRequiredLocalProps(rlp);
- toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD);
+ toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toAfterJoin.setLocalStrategy(LocalStrategy.NONE);
toAfterJoin.setRequiredGlobalProps(rgp);
@@ -1112,7 +1113,7 @@ public class FeedbackPropertiesMatchTest {
toJoin1.setRequiredGlobalProps(rgp1);
toJoin1.setRequiredLocalProps(rlp1);
- toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD);
+ toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toAfterJoin.setLocalStrategy(LocalStrategy.NONE);
toAfterJoin.setRequiredGlobalProps(rgp2);
@@ -1137,7 +1138,7 @@ public class FeedbackPropertiesMatchTest {
toJoin1.setRequiredGlobalProps(null);
toJoin1.setRequiredLocalProps(null);
- toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 1));
+ toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(2, 1), DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false});
toAfterJoin.setRequiredGlobalProps(rgp);
@@ -1159,7 +1160,7 @@ public class FeedbackPropertiesMatchTest {
toJoin1.setRequiredGlobalProps(null);
toJoin1.setRequiredLocalProps(null);
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false});
toAfterJoin.setRequiredGlobalProps(null);
@@ -1184,7 +1185,7 @@ public class FeedbackPropertiesMatchTest {
toJoin1.setRequiredGlobalProps(null);
toJoin1.setRequiredLocalProps(null);
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false});
toAfterJoin.setRequiredGlobalProps(rgp);
@@ -1206,27 +1207,27 @@ public class FeedbackPropertiesMatchTest {
SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Partial Solution");
Channel toMap1 = new Channel(target);
- toMap1.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap1.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map1 = new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP);
Channel toMap2 = new Channel(target);
- toMap2.setShipStrategy(ShipStrategyType.FORWARD);
+ toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toMap2.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode map2 = new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP);
Channel toJoin1 = new Channel(map1);
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
Channel toJoin2 = new Channel(map2);
- toJoin2.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin2.setLocalStrategy(LocalStrategy.NONE);
DualInputPlanNode join = new DualInputPlanNode(getJoinNode(), "Join", toJoin1, toJoin2, DriverStrategy.HYBRIDHASH_BUILD_FIRST);
Channel toAfterJoin = new Channel(join);
- toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD);
+ toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toAfterJoin.setLocalStrategy(LocalStrategy.NONE);
SingleInputPlanNode afterJoin = new SingleInputPlanNode(getMapNode(), "After Join Mapper", toAfterJoin, DriverStrategy.MAP);
@@ -1336,8 +1337,8 @@ public class FeedbackPropertiesMatchTest {
toJoin2.setRequiredGlobalProps(null);
toJoin2.setRequiredLocalProps(null);
- toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(88));
- toJoin2.setShipStrategy(ShipStrategyType.BROADCAST);
+ toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(88), DataExchangeMode.PIPELINED);
+ toJoin2.setShipStrategy(ShipStrategyType.BROADCAST, DataExchangeMode.PIPELINED);
toAfterJoin.setRequiredGlobalProps(rgp);
toAfterJoin.setRequiredLocalProps(rlp);
@@ -1358,8 +1359,8 @@ public class FeedbackPropertiesMatchTest {
RequestedLocalProperties rlp = new RequestedLocalProperties();
rlp.setGroupedFields(new FieldList(2, 1));
- toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(88));
- toJoin2.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(88), DataExchangeMode.PIPELINED);
+ toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toAfterJoin.setRequiredGlobalProps(rgp);
toAfterJoin.setRequiredLocalProps(rlp);
@@ -1380,8 +1381,8 @@ public class FeedbackPropertiesMatchTest {
RequestedLocalProperties rlp = new RequestedLocalProperties();
rlp.setGroupedFields(new FieldList(77, 69));
- toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(88));
- toJoin2.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.PARTITION_HASH, new FieldList(88), DataExchangeMode.PIPELINED);
+ toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toAfterJoin.setRequiredGlobalProps(rgp);
toAfterJoin.setRequiredLocalProps(rlp);
@@ -1400,10 +1401,10 @@ public class FeedbackPropertiesMatchTest {
rgp.setHashPartitioned(new FieldList(3));
- toJoin1.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.SORT, new FieldList(3), new boolean[] { false });
- toJoin2.setShipStrategy(ShipStrategyType.FORWARD);
+ toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
toJoin1.setLocalStrategy(LocalStrategy.NONE);
toAfterJoin.setRequiredGlobalProps(rgp);
@@ -1421,15 +1422,15 @@ public class FeedbackPropertiesMatchTest {
// --------------------------------------------------------------------------------------------
- private static final DataSourceNode getSourceNode() {
+ private static DataSourceNode getSourceNode() {
return new DataSourceNode(new GenericDataSourceBase<String, TextInputFormat>(new TextInputFormat(new Path("/")), new OperatorInformation<String>(BasicTypeInfo.STRING_TYPE_INFO)));
}
- private static final MapNode getMapNode() {
+ private static MapNode getMapNode() {
return new MapNode(new MapOperatorBase<String, String, MapFunction<String,String>>(new IdentityMapper<String>(), new UnaryOperatorInformation<String, String>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), "map op"));
}
- private static final JoinNode getJoinNode() {
+ private static JoinNode getJoinNode() {
return new JoinNode(new JoinOperatorBase<String, String, String, FlatJoinFunction<String, String, String>>(new DummyFlatJoinFunction<String>(), new BinaryOperatorInformation<String, String, String>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), new int[] {1}, new int[] {2}, "join op"));
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/costs/DefaultCostEstimatorTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/costs/DefaultCostEstimatorTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/costs/DefaultCostEstimatorTest.java
index 3c28a3a..01404ac 100644
--- a/flink-compiler/src/test/java/org/apache/flink/compiler/costs/DefaultCostEstimatorTest.java
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/costs/DefaultCostEstimatorTest.java
@@ -21,9 +21,6 @@ package org.apache.flink.compiler.costs;
import static org.junit.Assert.*;
-import org.apache.flink.compiler.costs.CostEstimator;
-import org.apache.flink.compiler.costs.Costs;
-import org.apache.flink.compiler.costs.DefaultCostEstimator;
import org.apache.flink.compiler.dag.EstimateProvider;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeClosedBranchingTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeClosedBranchingTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeClosedBranchingTest.java
new file mode 100644
index 0000000..a78336c
--- /dev/null
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeClosedBranchingTest.java
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.compiler.dataexchange;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.compiler.CompilerTestBase;
+import org.apache.flink.compiler.plan.DualInputPlanNode;
+import org.apache.flink.compiler.plan.OptimizedPlan;
+import org.apache.flink.compiler.plan.SingleInputPlanNode;
+import org.apache.flink.compiler.plan.SinkPlanNode;
+import org.apache.flink.compiler.testfunctions.DummyCoGroupFunction;
+import org.apache.flink.compiler.testfunctions.DummyFlatJoinFunction;
+import org.apache.flink.compiler.testfunctions.IdentityFlatMapper;
+import org.apache.flink.compiler.testfunctions.SelectOneReducer;
+import org.apache.flink.compiler.testfunctions.Top1GroupReducer;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
+import org.junit.Test;
+
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * This test checks the correct assignment of the DataExchangeMode to
+ * connections for programs that branch, and re-join those branches.
+ *
+ * <pre>
+ * /-> (sink)
+ * /
+ * /-> (reduce) -+ /-> (flatmap) -> (sink)
+ * / \ /
+ * (source) -> (map) - (join) -+-----\
+ * \ / \
+ * \-> (filter) -+ \
+ * \ (co group) -> (sink)
+ * \ /
+ * \-> (reduce) - /
+ * </pre>
+ */
+@SuppressWarnings("serial")
+public class DataExchangeModeClosedBranchingTest extends CompilerTestBase {
+
+ @Test
+ public void testPipelinedForced() {
+ // PIPELINED_FORCED should result in pipelining all the way
+ verifyBranchingJoiningPlan(ExecutionMode.PIPELINED_FORCED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED);
+ }
+
+ @Test
+ public void testPipelined() {
+ // PIPELINED should result in pipelining all the way
+ verifyBranchingJoiningPlan(ExecutionMode.PIPELINED,
+ DataExchangeMode.PIPELINED, // to map
+ DataExchangeMode.PIPELINED, // to combiner connections are pipelined
+ DataExchangeMode.BATCH, // to reduce
+ DataExchangeMode.BATCH, // to filter
+ DataExchangeMode.PIPELINED, // to sink after reduce
+ DataExchangeMode.PIPELINED, // to join (first input)
+ DataExchangeMode.BATCH, // to join (second input)
+ DataExchangeMode.PIPELINED, // combiner connections are pipelined
+ DataExchangeMode.BATCH, // to other reducer
+ DataExchangeMode.PIPELINED, // to flatMap
+ DataExchangeMode.PIPELINED, // to sink after flatMap
+ DataExchangeMode.PIPELINED, // to coGroup (first input)
+ DataExchangeMode.PIPELINED, // to coGroup (second input)
+ DataExchangeMode.PIPELINED // to sink after coGroup
+ );
+ }
+
+ @Test
+ public void testBatch() {
+ // BATCH should result in batching the shuffle all the way
+ verifyBranchingJoiningPlan(ExecutionMode.BATCH,
+ DataExchangeMode.PIPELINED, // to map
+ DataExchangeMode.PIPELINED, // to combiner connections are pipelined
+ DataExchangeMode.BATCH, // to reduce
+ DataExchangeMode.BATCH, // to filter
+ DataExchangeMode.PIPELINED, // to sink after reduce
+ DataExchangeMode.BATCH, // to join (first input)
+ DataExchangeMode.BATCH, // to join (second input)
+ DataExchangeMode.PIPELINED, // combiner connections are pipelined
+ DataExchangeMode.BATCH, // to other reducer
+ DataExchangeMode.PIPELINED, // to flatMap
+ DataExchangeMode.PIPELINED, // to sink after flatMap
+ DataExchangeMode.BATCH, // to coGroup (first input)
+ DataExchangeMode.BATCH, // to coGroup (second input)
+ DataExchangeMode.PIPELINED // to sink after coGroup
+ );
+ }
+
+ @Test
+ public void testBatchForced() {
+ // BATCH_FORCED should result in batching all the way
+ verifyBranchingJoiningPlan(ExecutionMode.BATCH_FORCED,
+ DataExchangeMode.BATCH, // to map
+ DataExchangeMode.PIPELINED, // to combiner connections are pipelined
+ DataExchangeMode.BATCH, // to reduce
+ DataExchangeMode.BATCH, // to filter
+ DataExchangeMode.BATCH, // to sink after reduce
+ DataExchangeMode.BATCH, // to join (first input)
+ DataExchangeMode.BATCH, // to join (second input)
+ DataExchangeMode.PIPELINED, // combiner connections are pipelined
+ DataExchangeMode.BATCH, // to other reducer
+ DataExchangeMode.BATCH, // to flatMap
+ DataExchangeMode.BATCH, // to sink after flatMap
+ DataExchangeMode.BATCH, // to coGroup (first input)
+ DataExchangeMode.BATCH, // to coGroup (second input)
+ DataExchangeMode.BATCH // to sink after coGroup
+ );
+ }
+
+ private void verifyBranchingJoiningPlan(ExecutionMode execMode,
+ DataExchangeMode toMap,
+ DataExchangeMode toReduceCombiner,
+ DataExchangeMode toReduce,
+ DataExchangeMode toFilter,
+ DataExchangeMode toReduceSink,
+ DataExchangeMode toJoin1,
+ DataExchangeMode toJoin2,
+ DataExchangeMode toOtherReduceCombiner,
+ DataExchangeMode toOtherReduce,
+ DataExchangeMode toFlatMap,
+ DataExchangeMode toFlatMapSink,
+ DataExchangeMode toCoGroup1,
+ DataExchangeMode toCoGroup2,
+ DataExchangeMode toCoGroupSink)
+ {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ env.getConfig().setExecutionMode(execMode);
+
+ DataSet<Tuple2<Long, Long>> data = env.fromElements(33L, 44L)
+ .map(new MapFunction<Long, Tuple2<Long, Long>>() {
+ @Override
+ public Tuple2<Long, Long> map(Long value) {
+ return new Tuple2<Long, Long>(value, value);
+ }
+ });
+
+ DataSet<Tuple2<Long, Long>> reduced = data.groupBy(0).reduce(new SelectOneReducer<Tuple2<Long, Long>>());
+ reduced.output(new DiscardingOutputFormat<Tuple2<Long, Long>>()).name("reduceSink");
+
+ DataSet<Tuple2<Long, Long>> filtered = data.filter(new FilterFunction<Tuple2<Long, Long>>() {
+ @Override
+ public boolean filter(Tuple2<Long, Long> value) throws Exception {
+ return false;
+ }
+ });
+
+ DataSet<Tuple2<Long, Long>> joined = reduced.join(filtered)
+ .where(1).equalTo(1)
+ .with(new DummyFlatJoinFunction<Tuple2<Long, Long>>());
+
+ joined.flatMap(new IdentityFlatMapper<Tuple2<Long, Long>>())
+ .output(new DiscardingOutputFormat<Tuple2<Long, Long>>()).name("flatMapSink");
+
+ joined.coGroup(filtered.groupBy(1).reduceGroup(new Top1GroupReducer<Tuple2<Long, Long>>()))
+ .where(0).equalTo(0)
+ .with(new DummyCoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>())
+ .output(new DiscardingOutputFormat<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>>()).name("cgSink");
+
+
+ OptimizedPlan optPlan = compileNoStats(env.createProgramPlan());
+
+ SinkPlanNode reduceSink = findSink(optPlan.getDataSinks(), "reduceSink");
+ SinkPlanNode flatMapSink = findSink(optPlan.getDataSinks(), "flatMapSink");
+ SinkPlanNode cgSink = findSink(optPlan.getDataSinks(), "cgSink");
+
+ DualInputPlanNode coGroupNode = (DualInputPlanNode) cgSink.getPredecessor();
+
+ DualInputPlanNode joinNode = (DualInputPlanNode) coGroupNode.getInput1().getSource();
+ SingleInputPlanNode otherReduceNode = (SingleInputPlanNode) coGroupNode.getInput2().getSource();
+ SingleInputPlanNode otherReduceCombinerNode = (SingleInputPlanNode) otherReduceNode.getPredecessor();
+
+ SingleInputPlanNode reduceNode = (SingleInputPlanNode) joinNode.getInput1().getSource();
+ SingleInputPlanNode reduceCombinerNode = (SingleInputPlanNode) reduceNode.getPredecessor();
+ assertEquals(reduceNode, reduceSink.getPredecessor());
+
+ SingleInputPlanNode filterNode = (SingleInputPlanNode) joinNode.getInput2().getSource();
+ assertEquals(filterNode, otherReduceCombinerNode.getPredecessor());
+
+ SingleInputPlanNode mapNode = (SingleInputPlanNode) filterNode.getPredecessor();
+ assertEquals(mapNode, reduceCombinerNode.getPredecessor());
+
+ SingleInputPlanNode flatMapNode = (SingleInputPlanNode) flatMapSink.getPredecessor();
+ assertEquals(joinNode, flatMapNode.getPredecessor());
+
+ // verify the data exchange modes
+
+ assertEquals(toReduceSink, reduceSink.getInput().getDataExchangeMode());
+ assertEquals(toFlatMapSink, flatMapSink.getInput().getDataExchangeMode());
+ assertEquals(toCoGroupSink, cgSink.getInput().getDataExchangeMode());
+
+ assertEquals(toCoGroup1, coGroupNode.getInput1().getDataExchangeMode());
+ assertEquals(toCoGroup2, coGroupNode.getInput2().getDataExchangeMode());
+
+ assertEquals(toJoin1, joinNode.getInput1().getDataExchangeMode());
+ assertEquals(toJoin2, joinNode.getInput2().getDataExchangeMode());
+
+ assertEquals(toOtherReduce, otherReduceNode.getInput().getDataExchangeMode());
+ assertEquals(toOtherReduceCombiner, otherReduceCombinerNode.getInput().getDataExchangeMode());
+
+ assertEquals(toFlatMap, flatMapNode.getInput().getDataExchangeMode());
+
+ assertEquals(toFilter, filterNode.getInput().getDataExchangeMode());
+ assertEquals(toReduce, reduceNode.getInput().getDataExchangeMode());
+ assertEquals(toReduceCombiner, reduceCombinerNode.getInput().getDataExchangeMode());
+
+ assertEquals(toMap, mapNode.getInput().getDataExchangeMode());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private SinkPlanNode findSink(Collection<SinkPlanNode> collection, String name) {
+ for (SinkPlanNode node : collection) {
+ String nodeName = node.getOptimizerNode().getPactContract().getName();
+ if (nodeName != null && nodeName.equals(name)) {
+ return node;
+ }
+ }
+
+ throw new IllegalArgumentException("No node with that name was found.");
+ }
+}
[3/8] flink git commit: [FLINK-1671] [tests] Rename "ExecutionMode"
in MultiProgramTestBase to "TestExecutionMode"
Posted by se...@apache.org.
[FLINK-1671] [tests] Rename "ExecutionMode" in MultiProgramTestBase to "TestExecutionMode"
... to avoid name clashes with the program execution mode.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/47827702
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/47827702
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/47827702
Branch: refs/heads/master
Commit: 47827702658e8ce8a13a3fedda76303ac1d5fb85
Parents: 361c90a
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Mar 10 18:36:52 2015 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Mar 17 10:10:08 2015 +0100
----------------------------------------------------------------------
.../apache/flink/api/io/avro/AvroPojoTest.java | 2 +-
.../scala/expressions/AggregationsITCase.scala | 4 +-
.../flink/api/scala/expressions/AsITCase.scala | 4 +-
.../api/scala/expressions/CastingITCase.scala | 4 +-
.../scala/expressions/ExpressionsITCase.scala | 4 +-
.../api/scala/expressions/FilterITCase.scala | 4 +-
.../GroupedAggreagationsITCase.scala | 4 +-
.../api/scala/expressions/JoinITCase.scala | 4 +-
.../api/scala/expressions/SelectITCase.scala | 4 +-
.../expressions/StringExpressionsITCase.scala | 4 +-
.../flink/graph/test/WeaklyConnectedITCase.java | 2 +-
.../example/LabelPropagationExampleITCase.java | 2 +-
.../SingleSourceShortestPathsITCase.java | 2 +-
.../graph/test/operations/DegreesITCase.java | 2 +-
.../test/operations/FromCollectionITCase.java | 2 +-
.../test/operations/GraphCreationITCase.java | 2 +-
.../GraphCreationWithMapperITCase.java | 2 +-
.../test/operations/GraphMutationsITCase.java | 2 +-
.../test/operations/GraphOperationsITCase.java | 2 +-
.../test/operations/JoinWithEdgesITCase.java | 2 +-
.../test/operations/JoinWithVerticesITCase.java | 2 +-
.../graph/test/operations/MapEdgesITCase.java | 2 +-
.../test/operations/MapVerticesITCase.java | 2 +-
.../operations/ReduceOnEdgesMethodsITCase.java | 2 +-
.../ReduceOnNeighborMethodsITCase.java | 2 +-
.../mapred/HadoopMapFunctionITCase.java | 2 +-
.../HadoopReduceCombineFunctionITCase.java | 4 +-
.../mapred/HadoopReduceFunctionITCase.java | 2 +-
.../test/util/MultipleProgramsTestBase.java | 59 ++++++++++++++++----
.../apache/flink/test/util/TestBaseUtils.java | 8 ++-
.../flink/test/actions/CountCollectITCase.java | 2 +-
.../exampleJavaPrograms/PageRankITCase.java | 2 +-
.../exampleScalaPrograms/PageRankITCase.java | 2 +-
.../aggregators/AggregatorsITCase.java | 2 +-
.../test/javaApiOperators/AggregateITCase.java | 2 +-
.../test/javaApiOperators/CoGroupITCase.java | 2 +-
.../test/javaApiOperators/CrossITCase.java | 2 +-
.../test/javaApiOperators/DataSinkITCase.java | 2 +-
.../test/javaApiOperators/DistinctITCase.java | 2 +-
.../test/javaApiOperators/FilterITCase.java | 2 +-
.../test/javaApiOperators/FirstNITCase.java | 2 +-
.../test/javaApiOperators/FlatMapITCase.java | 2 +-
.../javaApiOperators/GroupReduceITCase.java | 10 ++--
.../flink/test/javaApiOperators/JoinITCase.java | 2 +-
.../flink/test/javaApiOperators/MapITCase.java | 2 +-
.../test/javaApiOperators/PartitionITCase.java | 2 +-
.../test/javaApiOperators/ReduceITCase.java | 2 +-
.../ReplicatingDataSourceITCase.java | 3 +-
.../javaApiOperators/SortPartitionITCase.java | 2 +-
.../test/javaApiOperators/SumMinMaxITCase.java | 2 +-
.../test/javaApiOperators/UnionITCase.java | 2 +-
.../api/scala/actions/CountCollectITCase.scala | 4 +-
.../scala/functions/ClosureCleanerITCase.scala | 4 +-
.../api/scala/operators/AggregateITCase.scala | 4 +-
.../api/scala/operators/CoGroupITCase.scala | 4 +-
.../flink/api/scala/operators/CrossITCase.scala | 4 +-
.../api/scala/operators/DistinctITCase.scala | 4 +-
.../api/scala/operators/ExamplesITCase.scala | 4 +-
.../api/scala/operators/FilterITCase.scala | 4 +-
.../api/scala/operators/FirstNITCase.scala | 4 +-
.../api/scala/operators/FlatMapITCase.scala | 4 +-
.../api/scala/operators/GroupReduceITCase.scala | 12 ++--
.../flink/api/scala/operators/JoinITCase.scala | 4 +-
.../flink/api/scala/operators/MapITCase.scala | 4 +-
.../api/scala/operators/PartitionITCase.scala | 8 +--
.../api/scala/operators/ReduceITCase.scala | 4 +-
.../api/scala/operators/SumMinMaxITCase.scala | 4 +-
.../flink/api/scala/operators/UnionITCase.scala | 4 +-
.../scala/runtime/ScalaSpecialTypesITCase.scala | 4 +-
69 files changed, 159 insertions(+), 119 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
index f367ed1..bddb79c 100644
--- a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
+++ b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/io/avro/AvroPojoTest.java
@@ -43,7 +43,7 @@ import java.util.Arrays;
@RunWith(Parameterized.class)
public class AvroPojoTest extends MultipleProgramsTestBase {
- public AvroPojoTest(ExecutionMode mode) {
+ public AvroPojoTest(TestExecutionMode mode) {
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AggregationsITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AggregationsITCase.scala
index a0d2005..fee7ac8 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AggregationsITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AggregationsITCase.scala
@@ -23,14 +23,14 @@ import org.apache.flink.api.scala._
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class AggregationsITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AsITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AsITCase.scala
index 55969c4..8921e89 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AsITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/AsITCase.scala
@@ -24,7 +24,7 @@ import org.apache.flink.api.scala._
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
@@ -33,7 +33,7 @@ import org.junit.runners.Parameterized
import scala.collection.JavaConverters._
@RunWith(classOf[Parameterized])
-class AsITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class AsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/CastingITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/CastingITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/CastingITCase.scala
index 1515fb5..b3f8ef3 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/CastingITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/CastingITCase.scala
@@ -23,14 +23,14 @@ import org.apache.flink.api.scala._
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class CastingITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class CastingITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/ExpressionsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/ExpressionsITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/ExpressionsITCase.scala
index 46074f3..de41f65 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/ExpressionsITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/ExpressionsITCase.scala
@@ -22,14 +22,14 @@ import org.apache.flink.api.expressions.ExpressionException
import org.apache.flink.api.scala._
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class ExpressionsITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class ExpressionsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/FilterITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/FilterITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/FilterITCase.scala
index 04937f2..4b46458 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/FilterITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/FilterITCase.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.scala._
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@@ -31,7 +31,7 @@ import org.junit._
@RunWith(classOf[Parameterized])
-class FilterITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class FilterITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/GroupedAggreagationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/GroupedAggreagationsITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/GroupedAggreagationsITCase.scala
index 718ea6a..06f61db 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/GroupedAggreagationsITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/GroupedAggreagationsITCase.scala
@@ -23,14 +23,14 @@ import org.apache.flink.api.scala._
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class GroupedAggreagationsITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class GroupedAggreagationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/JoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/JoinITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/JoinITCase.scala
index 2cc37ff..d52acf6 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/JoinITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/JoinITCase.scala
@@ -24,14 +24,14 @@ import org.apache.flink.api.scala._
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class JoinITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/SelectITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/SelectITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/SelectITCase.scala
index 6acb016..b286421 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/SelectITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/SelectITCase.scala
@@ -23,14 +23,14 @@ import org.apache.flink.api.scala._
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class SelectITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class SelectITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/StringExpressionsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/StringExpressionsITCase.scala b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/StringExpressionsITCase.scala
index b04d4dd..3a7ad02 100644
--- a/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/StringExpressionsITCase.scala
+++ b/flink-staging/flink-expressions/src/test/scala/org/apache/flink/api/scala/expressions/StringExpressionsITCase.scala
@@ -22,14 +22,14 @@ import org.apache.flink.api.expressions.ExpressionException
import org.apache.flink.api.scala._
import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit._
import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class StringExpressionsITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class StringExpressionsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = ""
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/WeaklyConnectedITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/WeaklyConnectedITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/WeaklyConnectedITCase.java
index dd08f47..1628952 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/WeaklyConnectedITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/WeaklyConnectedITCase.java
@@ -32,7 +32,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class WeaklyConnectedITCase extends MultipleProgramsTestBase {
- public WeaklyConnectedITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public WeaklyConnectedITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationExampleITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationExampleITCase.java
index 185d922..0e1810f 100755
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationExampleITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/LabelPropagationExampleITCase.java
@@ -35,7 +35,7 @@ import java.io.File;
@RunWith(Parameterized.class)
public class LabelPropagationExampleITCase extends MultipleProgramsTestBase {
- public LabelPropagationExampleITCase(ExecutionMode mode){
+ public LabelPropagationExampleITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
index 761d71f..9f563da 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
@@ -47,7 +47,7 @@ public class SingleSourceShortestPathsITCase extends MultipleProgramsTestBase {
@Rule
public TemporaryFolder tempFolder = new TemporaryFolder();
- public SingleSourceShortestPathsITCase(ExecutionMode mode) {
+ public SingleSourceShortestPathsITCase(TestExecutionMode mode) {
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
index 8c363a5..0391dcb 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
@@ -34,7 +34,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class DegreesITCase extends MultipleProgramsTestBase {
- public DegreesITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public DegreesITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
index 975d21a..5d4b7d7 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
@@ -35,7 +35,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class FromCollectionITCase extends MultipleProgramsTestBase {
- public FromCollectionITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public FromCollectionITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
index 6848dad..dc0e5d2 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
@@ -40,7 +40,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class GraphCreationITCase extends MultipleProgramsTestBase {
- public GraphCreationITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public GraphCreationITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
index 010ae1d..23d05ac 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
@@ -36,7 +36,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class GraphCreationWithMapperITCase extends MultipleProgramsTestBase {
- public GraphCreationWithMapperITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public GraphCreationWithMapperITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
index 502d529..dce34a8 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
@@ -38,7 +38,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class GraphMutationsITCase extends MultipleProgramsTestBase {
- public GraphMutationsITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public GraphMutationsITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
index 6c4f1ef..e7f067a 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
@@ -39,7 +39,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class GraphOperationsITCase extends MultipleProgramsTestBase {
- public GraphOperationsITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public GraphOperationsITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
index dfb315e..c02f0bb 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
@@ -39,7 +39,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class JoinWithEdgesITCase extends MultipleProgramsTestBase {
- public JoinWithEdgesITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public JoinWithEdgesITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
index 28a0441..120e97a 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
@@ -38,7 +38,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class JoinWithVerticesITCase extends MultipleProgramsTestBase {
- public JoinWithVerticesITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public JoinWithVerticesITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
index a5c01cf..d1ba9a5 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
@@ -39,7 +39,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class MapEdgesITCase extends MultipleProgramsTestBase {
- public MapEdgesITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public MapEdgesITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
index 0d92fc9..f3a63be 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
@@ -39,7 +39,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class MapVerticesITCase extends MultipleProgramsTestBase {
- public MapVerticesITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public MapVerticesITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
index 73dec2a..ec0c84c 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
@@ -40,7 +40,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class ReduceOnEdgesMethodsITCase extends MultipleProgramsTestBase {
- public ReduceOnEdgesMethodsITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public ReduceOnEdgesMethodsITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
index c1e982f..785552c 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
@@ -43,7 +43,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class ReduceOnNeighborMethodsITCase extends MultipleProgramsTestBase {
- public ReduceOnNeighborMethodsITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public ReduceOnNeighborMethodsITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
index 850e799..4d1acb4 100644
--- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
+++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
@@ -41,7 +41,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class HadoopMapFunctionITCase extends MultipleProgramsTestBase {
- public HadoopMapFunctionITCase(ExecutionMode mode){
+ public HadoopMapFunctionITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
index 92b0dc3..13d971c 100644
--- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
+++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
@@ -44,7 +44,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class HadoopReduceCombineFunctionITCase extends MultipleProgramsTestBase {
- public HadoopReduceCombineFunctionITCase(ExecutionMode mode){
+ public HadoopReduceCombineFunctionITCase(TestExecutionMode mode){
super(mode);
}
@@ -99,7 +99,7 @@ public class HadoopReduceCombineFunctionITCase extends MultipleProgramsTestBase
@Test
public void testCombiner() throws Exception {
- org.junit.Assume.assumeThat(mode, new IsEqual<ExecutionMode>(ExecutionMode.CLUSTER));
+ org.junit.Assume.assumeThat(mode, new IsEqual<TestExecutionMode>(TestExecutionMode.CLUSTER));
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<IntWritable, IntWritable>> ds = HadoopTestData.getKVPairDataSet(env).
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
index 1801c3e..abc0e9c 100644
--- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
+++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
@@ -42,7 +42,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class HadoopReduceFunctionITCase extends MultipleProgramsTestBase {
- public HadoopReduceFunctionITCase(ExecutionMode mode){
+ public HadoopReduceFunctionITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
index c86ccaa..850cb88 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
@@ -25,11 +25,54 @@ import org.junit.runners.Parameterized;
import java.util.Arrays;
import java.util.Collection;
+/**
+ * Base class for unit tests that run multiple tests and want to reuse the same
+ * Flink cluster. This saves a significant amount of time, since the startup and
+ * shutdown of the Flink clusters (including actor systems, etc) usually dominates
+ * the execution of the actual tests.
+ *
+ * To write a unit test against this test base, simply extend it and add
+ * one or more regular test methods and retrieve the ExecutionEnvironment from
+ * the context:
+ *
+ * <pre>{@code
+ *
+ * @Test
+ * public void someTest() {
+ * ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ * // test code
+ * env.execute();
+ * }
+ *
+ * @Test
+ * public void anotherTest() {
+ * ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ * // test code
+ * env.execute();
+ * }
+ *
+ * }</pre>
+ */
public class MultipleProgramsTestBase extends TestBaseUtils {
+
+ /**
+ * Enum that defines which execution environment to run the next test on:
+ * An embedded local flink cluster, or the collection execution backend.
+ */
+ public static enum TestExecutionMode {
+ CLUSTER,
+ COLLECTION
+ }
+
+ // -----------------------------------------------------------------------------------------...
+
+ private static final int DEFAULT_PARALLELISM = 4;
+
protected static ForkableFlinkMiniCluster cluster = null;
- protected transient ExecutionMode mode;
- public MultipleProgramsTestBase(ExecutionMode mode){
+ protected transient TestExecutionMode mode;
+
+ public MultipleProgramsTestBase(TestExecutionMode mode){
this.mode = mode;
switch(mode){
case CLUSTER:
@@ -45,7 +88,7 @@ public class MultipleProgramsTestBase extends TestBaseUtils {
@BeforeClass
public static void setup() throws Exception{
- cluster = TestBaseUtils.startCluster(1, 4);
+ cluster = TestBaseUtils.startCluster(1, DEFAULT_PARALLELISM);
}
@AfterClass
@@ -54,12 +97,8 @@ public class MultipleProgramsTestBase extends TestBaseUtils {
}
@Parameterized.Parameters(name = "Execution mode = {0}")
- public static Collection<ExecutionMode[]> executionModes(){
- return Arrays.asList(new ExecutionMode[]{ExecutionMode.CLUSTER},
- new ExecutionMode[]{ExecutionMode.COLLECTION});
- }
-
- protected static enum ExecutionMode{
- CLUSTER, COLLECTION
+ public static Collection<TestExecutionMode[]> executionModes(){
+ return Arrays.asList(new TestExecutionMode[]{TestExecutionMode.CLUSTER},
+ new TestExecutionMode[]{TestExecutionMode.COLLECTION});
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index 804b005..64dd3f1 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -80,8 +80,9 @@ public class TestBaseUtils {
+ "m", heap > MINIMUM_HEAP_SIZE_MB - 50);
}
- protected static ForkableFlinkMiniCluster startCluster(int numTaskManagers, int
- taskManagerNumSlots) throws Exception {
+ protected static ForkableFlinkMiniCluster startCluster(int numTaskManagers, int taskManagerNumSlots)
+ throws Exception {
+
Configuration config = new Configuration();
config.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, true);
config.setBoolean(ConfigConstants.TASK_MANAGER_MEMORY_LAZY_ALLOCATION_KEY, true);
@@ -95,7 +96,8 @@ public class TestBaseUtils {
protected static void stopCluster(ForkableFlinkMiniCluster executor, FiniteDuration timeout)
throws Exception {
- if(executor != null) {
+
+ if (executor != null) {
int numUnreleasedBCVars = 0;
int numActiveConnections = 0;
{
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java b/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
index b432105..e742c27 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
@@ -38,7 +38,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class CountCollectITCase extends MultipleProgramsTestBase {
- public CountCollectITCase(ExecutionMode mode) {
+ public CountCollectITCase(TestExecutionMode mode) {
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
index 0c3ad4a..1c66c3e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
@@ -36,7 +36,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class PageRankITCase extends MultipleProgramsTestBase {
- public PageRankITCase(ExecutionMode mode){
+ public PageRankITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
index 6fe549f..f9c2566 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
@@ -36,7 +36,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class PageRankITCase extends MultipleProgramsTestBase {
- public PageRankITCase(ExecutionMode mode){
+ public PageRankITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
----------------------------------------------------------------------
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 9dcf6fc..5aa6b42 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
@@ -54,7 +54,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
private static final int DOP = 2;
private static final String NEGATIVE_ELEMENTS_AGGR = "count.negative.elements";
- public AggregatorsITCase(ExecutionMode mode){
+ public AggregatorsITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
index 3fbcae6..ea7fc5a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
@@ -38,7 +38,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
public class AggregateITCase extends MultipleProgramsTestBase {
- public AggregateITCase(ExecutionMode mode){
+ public AggregateITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
index b249e22..99f568e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
@@ -50,7 +50,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class CoGroupITCase extends MultipleProgramsTestBase {
- public CoGroupITCase(ExecutionMode mode){
+ public CoGroupITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
index bd32bfc..74868a0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
@@ -43,7 +43,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
@RunWith(Parameterized.class)
public class CrossITCase extends MultipleProgramsTestBase {
- public CrossITCase(ExecutionMode mode){
+ public CrossITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java
----------------------------------------------------------------------
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 6bd678f..dbf4798 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
@@ -45,7 +45,7 @@ import static org.junit.Assert.assertTrue;
@RunWith(Parameterized.class)
public class DataSinkITCase extends MultipleProgramsTestBase {
- public DataSinkITCase(ExecutionMode mode) {
+ public DataSinkITCase(TestExecutionMode mode) {
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
index 84964a2..4568ab6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
@@ -44,7 +44,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class DistinctITCase extends MultipleProgramsTestBase {
- public DistinctITCase(ExecutionMode mode){
+ public DistinctITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
index 3bf83b4..c46bc46 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
@@ -39,7 +39,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
@RunWith(Parameterized.class)
public class FilterITCase extends MultipleProgramsTestBase {
- public FilterITCase(ExecutionMode mode){
+ public FilterITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java
index 24bc3e6..feb1169 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java
@@ -37,7 +37,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class FirstNITCase extends MultipleProgramsTestBase {
- public FirstNITCase(ExecutionMode mode){
+ public FirstNITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
index bf49eae..37cf1fc 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
@@ -40,7 +40,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
@RunWith(Parameterized.class)
public class FlatMapITCase extends MultipleProgramsTestBase {
- public FlatMapITCase(ExecutionMode mode){
+ public FlatMapITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
----------------------------------------------------------------------
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 02ef7a7..95e1dec 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
@@ -57,7 +57,7 @@ import scala.math.BigInt;
@RunWith(Parameterized.class)
public class GroupReduceITCase extends MultipleProgramsTestBase {
- public GroupReduceITCase(ExecutionMode mode){
+ public GroupReduceITCase(TestExecutionMode mode){
super(mode);
}
@@ -309,7 +309,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
/*
* check correctness of groupReduce on custom type with key extractor and combine
*/
- org.junit.Assume.assumeTrue(mode != ExecutionMode.COLLECTION);
+ org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -343,7 +343,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
/*
* check correctness of groupReduce on tuples with combine
*/
- org.junit.Assume.assumeTrue(mode != ExecutionMode.COLLECTION);
+ 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
@@ -369,7 +369,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
/*
* check correctness of all-groupreduce for tuples with combine
*/
- org.junit.Assume.assumeTrue(mode != ExecutionMode.COLLECTION);
+ org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -842,7 +842,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
env.execute();
// return expected result
- if (super.mode == ExecutionMode.COLLECTION) {
+ if (super.mode == TestExecutionMode.COLLECTION) {
expected = null;
} else {
expected = "1,Hi\n" +
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
----------------------------------------------------------------------
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 f10a9df..375baee 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
@@ -49,7 +49,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
@RunWith(Parameterized.class)
public class JoinITCase extends MultipleProgramsTestBase {
- public JoinITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public JoinITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
index 4fbb53d..083f0a3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
@@ -42,7 +42,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
@RunWith(Parameterized.class)
public class MapITCase extends MultipleProgramsTestBase {
- public MapITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public MapITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java
----------------------------------------------------------------------
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 cf78a34..e1603ca 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
@@ -45,7 +45,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class PartitionITCase extends MultipleProgramsTestBase {
- public PartitionITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public PartitionITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
index 1168e3c..0232464 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
@@ -50,7 +50,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class ReduceITCase extends MultipleProgramsTestBase {
- public ReduceITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public ReduceITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java
index 85e3e11..c7ca37d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java
@@ -33,7 +33,6 @@ import org.apache.flink.core.io.GenericInputSplit;
import org.apache.flink.test.util.MultipleProgramsTestBase;
import org.apache.flink.util.NumberSequenceIterator;
import org.junit.After;
-import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
@@ -48,7 +47,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class ReplicatingDataSourceITCase extends MultipleProgramsTestBase {
- public ReplicatingDataSourceITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public ReplicatingDataSourceITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java
----------------------------------------------------------------------
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 790b7ba..4bba558 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
@@ -45,7 +45,7 @@ import java.util.Iterator;
@RunWith(Parameterized.class)
public class SortPartitionITCase extends MultipleProgramsTestBase {
- public SortPartitionITCase(ExecutionMode mode){
+ public SortPartitionITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java
index ee12fa4..e6367c3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java
@@ -36,7 +36,7 @@ import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class SumMinMaxITCase extends MultipleProgramsTestBase {
- public SumMinMaxITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public SumMinMaxITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
index 0a23da8..2e7ae9c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
@@ -57,7 +57,7 @@ public class UnionITCase extends MultipleProgramsTestBase {
"20,6,Comment#14\n" +
"21,6,Comment#15\n";
- public UnionITCase(MultipleProgramsTestBase.ExecutionMode mode){
+ public UnionITCase(TestExecutionMode mode){
super(mode);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/actions/CountCollectITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/actions/CountCollectITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/actions/CountCollectITCase.scala
index a1bbd2e..6e95b7b 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/actions/CountCollectITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/actions/CountCollectITCase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala.actions
import org.apache.flink.api.scala._
import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.junit.Test
import org.junit.Assert._
@@ -28,7 +28,7 @@ import org.junit.runner.RunWith
import org.junit.runners.Parameterized
@RunWith(classOf[Parameterized])
-class CountCollectITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class CountCollectITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
@Test
def testCountCollectOnSimpleJob(): Unit = {
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/ClosureCleanerITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/ClosureCleanerITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/ClosureCleanerITCase.scala
index 5056b82..2cae79c 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/ClosureCleanerITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/functions/ClosureCleanerITCase.scala
@@ -19,7 +19,7 @@ package org.apache.flink.api.scala.functions
import org.apache.flink.api.scala.ExecutionEnvironment
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.MultipleProgramsTestBase
import org.junit.Assert.fail
import org.junit.{After, Before, Test, Rule}
@@ -33,7 +33,7 @@ import org.apache.flink.api.common.InvalidProgramException
/* The test cases are originally from the Apache Spark project. Like the ClosureCleaner itself. */
@RunWith(classOf[Parameterized])
-class ClosureCleanerITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class ClosureCleanerITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
val _tempFolder = new TemporaryFolder()
var resultPath: String = _
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/AggregateITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/AggregateITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/AggregateITCase.scala
index 8839093..484226d 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/AggregateITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/AggregateITCase.scala
@@ -21,7 +21,7 @@ import org.apache.flink.api.java.aggregation.Aggregations
import org.apache.flink.api.scala.ExecutionEnvironment
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -31,7 +31,7 @@ import org.junit.runners.Parameterized
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class AggregateITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class AggregateITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala
index e90cb1b..42ec42b 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.api.scala.util.CollectionDataSets.CustomType
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.apache.flink.util.Collector
import org.junit.rules.TemporaryFolder
@@ -35,7 +35,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class CoGroupITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class CoGroupITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
val _tempFolder = new TemporaryFolder()
var resultPath: String = _
var expectedResult: String = _
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala
index 17b8252..a8611c8 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.api.scala.util.CollectionDataSets.CustomType
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -34,7 +34,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class CrossITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class CrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/DistinctITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/DistinctITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/DistinctITCase.scala
index 75cf676..31c6052 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/DistinctITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/DistinctITCase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala.operators
import org.apache.flink.api.scala.ExecutionEnvironment
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -30,7 +30,7 @@ import org.junit.runners.Parameterized
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class DistinctITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class DistinctITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ExamplesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ExamplesITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ExamplesITCase.scala
index 58ce51b..2bccc5b 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ExamplesITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ExamplesITCase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala.operators
import org.apache.flink.api.scala.ExecutionEnvironment
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase, JavaProgramTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -62,7 +62,7 @@ class PojoWithPojo(var myString: String, var myInt: Int, var nested: Nested) {
}
@RunWith(classOf[Parameterized])
-class ExamplesITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class ExamplesITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala
index d0a4ee4..8336ae3 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala
@@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.RichFilterFunction
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -33,7 +33,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class FilterITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class FilterITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FirstNITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FirstNITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FirstNITCase.scala
index 3409b48..9c59206 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FirstNITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FirstNITCase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala.operators
import org.apache.flink.api.common.operators.Order
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -30,7 +30,7 @@ import org.junit.runners.Parameterized
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class FirstNITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class FirstNITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala
index 71ad28f..4a66b80 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.apache.flink.util.Collector
import org.junit.{Test, After, Before, Rule}
@@ -35,7 +35,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class FlatMapITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class FlatMapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala
----------------------------------------------------------------------
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 3f62133..1d86f77 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
@@ -28,7 +28,7 @@ CustomType}
import org.apache.flink.compiler.PactCompiler
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.apache.flink.util.Collector
import org.hamcrest.core.{IsNot, IsEqual}
@@ -42,7 +42,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class GroupReduceITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
@@ -277,7 +277,7 @@ class GroupReduceITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mo
/*
* check correctness of groupReduce on custom type with key extractor and combine
*/
- org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(ExecutionMode.COLLECTION)))
+ org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(TestExecutionMode.COLLECTION)))
val env = ExecutionEnvironment.getExecutionEnvironment
val ds = CollectionDataSets.getCustomTypeDataSet(env)
@@ -297,7 +297,7 @@ class GroupReduceITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mo
/*
* check correctness of groupReduce on tuples with combine
*/
- org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(ExecutionMode.COLLECTION)))
+ org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(TestExecutionMode.COLLECTION)))
val env = ExecutionEnvironment.getExecutionEnvironment
// important because it determines how often the combiner is called
@@ -318,7 +318,7 @@ class GroupReduceITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mo
/*
* check correctness of all-groupreduce for tuples with combine
*/
- org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(ExecutionMode.COLLECTION)))
+ org.junit.Assume.assumeThat(mode, new IsNot(new IsEqual(TestExecutionMode.COLLECTION)))
val env = ExecutionEnvironment.getExecutionEnvironment
val ds = CollectionDataSets.get3TupleDataSet(env).map(t => t).setParallelism(4)
@@ -665,7 +665,7 @@ class GroupReduceITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mo
.reduceGroup(new Tuple3SortedGroupReduceWithCombine)
reduceDs.writeAsCsv(resultPath)
env.execute()
- if (mode == ExecutionMode.COLLECTION) {
+ if (mode == TestExecutionMode.COLLECTION) {
expected = null
} else {
expected = "1,Hi\n" +
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala
----------------------------------------------------------------------
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 cf13a42..c5ced67 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
@@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.RichJoinFunction
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -33,7 +33,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class JoinITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala
index 4c74ca0..5ade21f 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit._
import org.junit.rules.TemporaryFolder
@@ -34,7 +34,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class MapITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class MapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/PartitionITCase.scala
----------------------------------------------------------------------
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 f21fe11..8e00075 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
@@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.{RichFilterFunction, RichMapFunctio
import org.apache.flink.api.scala.ExecutionEnvironment
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -31,7 +31,7 @@ import org.junit.runners.Parameterized
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class PartitionITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
@@ -153,7 +153,7 @@ class PartitionITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode
count.writeAsText(resultPath, WriteMode.OVERWRITE)
env.execute()
- expected = if (mode == ExecutionMode.COLLECTION) "(1)\n" else "(4)\n"
+ expected = if (mode == TestExecutionMode.COLLECTION) "(1)\n" else "(4)\n"
}
@Test
@@ -182,7 +182,7 @@ class PartitionITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode
count.writeAsText(resultPath, WriteMode.OVERWRITE)
env.execute()
- expected = if (mode == ExecutionMode.COLLECTION) "(1)\n" else "(4)\n"
+ expected = if (mode == TestExecutionMode.COLLECTION) "(1)\n" else "(4)\n"
}
@Test
[4/8] flink git commit: [FLINK-1671] [core] Add execution mode to
execution config
Posted by se...@apache.org.
[FLINK-1671] [core] Add execution mode to execution config
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7f8e1566
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7f8e1566
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7f8e1566
Branch: refs/heads/master
Commit: 7f8e156617ae68d8339249ee276bcc36552a49da
Parents: 4782770
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Mar 10 20:25:01 2015 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Mar 17 10:10:08 2015 +0100
----------------------------------------------------------------------
.../org/apache/flink/compiler/PactCompiler.java | 78 +++++-------
.../flink/compiler/dag/PactConnection.java | 51 ++++++--
.../flink/api/common/ExecutionConfig.java | 127 ++++++++++++++-----
.../apache/flink/api/common/ExecutionMode.java | 83 ++++++++++++
4 files changed, 251 insertions(+), 88 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/7f8e1566/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java b/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
index a82bb74..d5cbf68 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
@@ -353,25 +353,18 @@ public class PactCompiler {
// ------------------------------------------------------------------------
/**
- * Creates a new compiler instance. The compiler has no access to statistics about the
+ * Creates a new optimizer instance. The optimizer has no access to statistics about the
* inputs and can hence not determine any properties. It will perform all optimization with
- * unknown sizes and default to the most robust execution strategies. The
- * compiler also uses conservative default estimates for the operator costs, since
- * it has no access to another cost estimator.
- * <p>
- * The address of the job manager (to obtain system characteristics) is determined via the global configuration.
+ * unknown sizes and hence use only the heuristic cost functions, which result in the selection
+ * of the most robust execution strategies.
*/
public PactCompiler() {
this(null, new DefaultCostEstimator());
}
/**
- * Creates a new compiler instance that uses the statistics object to determine properties about the input.
- * Given those statistics, the compiler can make better choices for the execution strategies.
- * as if no filesystem was given. The compiler uses conservative default estimates for the operator costs, since
- * it has no access to another cost estimator.
- * <p>
- * The address of the job manager (to obtain system characteristics) is determined via the global configuration.
+ * Creates a new optimizer instance that uses the statistics object to determine properties about the input.
+ * Given those statistics, the optimizer can make better choices for the execution strategies.
*
* @param stats
* The statistics to be used to determine the input properties.
@@ -381,27 +374,24 @@ public class PactCompiler {
}
/**
- * Creates a new compiler instance. The compiler has no access to statistics about the
+ * Creates a new optimizer instance. The optimizer has no access to statistics about the
* inputs and can hence not determine any properties. It will perform all optimization with
- * unknown sizes and default to the most robust execution strategies. It uses
- * however the given cost estimator to compute the costs of the individual operations.
- * <p>
- * The address of the job manager (to obtain system characteristics) is determined via the global configuration.
+ * unknown sizes and hence use only the heuristic cost functions, which result in the selection
+ * of the most robust execution strategies.
+ *
+ * The optimizer uses the given cost estimator to compute the costs of the individual operations.
*
- * @param estimator
- * The <tt>CostEstimator</tt> to use to cost the individual operations.
+ * @param estimator The cost estimator to use to cost the individual operations.
*/
public PactCompiler(CostEstimator estimator) {
this(null, estimator);
}
/**
- * Creates a new compiler instance that uses the statistics object to determine properties about the input.
- * Given those statistics, the compiler can make better choices for the execution strategies.
- * as if no filesystem was given. It uses the given cost estimator to compute the costs of the individual
- * operations.
- * <p>
- * The address of the job manager (to obtain system characteristics) is determined via the global configuration.
+ * Creates a new optimizer instance that uses the statistics object to determine properties about the input.
+ * Given those statistics, the optimizer can make better choices for the execution strategies.
+ *
+ * The optimizer uses the given cost estimator to compute the costs of the individual operations.
*
* @param stats
* The statistics to be used to determine the input properties.
@@ -412,9 +402,10 @@ public class PactCompiler {
this.statistics = stats;
this.costEstimator = estimator;
- // determine the default parallelization degree
- this.defaultDegreeOfParallelism = GlobalConfiguration.getInteger(ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY,
- ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE);
+ // determine the default parallelism
+ this.defaultDegreeOfParallelism = GlobalConfiguration.getInteger(
+ ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY,
+ ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE);
if (defaultDegreeOfParallelism < 1) {
LOG.warn("Config value " + defaultDegreeOfParallelism + " for option "
@@ -444,36 +435,35 @@ public class PactCompiler {
// ------------------------------------------------------------------------
/**
- * Translates the given plan in to an OptimizedPlan, where all nodes have their local strategy assigned
- * and all channels have a shipping strategy assigned. The compiler connects to the job manager to obtain information
- * about the available instances and their memory and then chooses an instance type to schedule the execution on.
- * <p>
- * The compilation process itself goes through several phases:
- * <ol>
- * <li>Create an optimizer data flow representation of the program, assign parallelism and compute size estimates.</li>
- * <li>Compute interesting properties and auxiliary structures.</li>
- * <li>Enumerate plan alternatives. This cannot be done in the same step as the interesting property computation (as
- * opposed to the Database approaches), because we support plans that are not trees.</li>
- * </ol>
+ * Translates the given program to an OptimizedPlan, where all nodes have their local strategy assigned
+ * and all channels have a shipping strategy assigned.
+ *
+ * For more details on the optimization phase, see the comments for
+ * {@link #compile(org.apache.flink.api.common.Plan, org.apache.flink.compiler.postpass.OptimizerPostPass)}.
*
* @param program The program to be translated.
* @return The optimized plan.
+ *
* @throws CompilerException
* Thrown, if the plan is invalid or the optimizer encountered an inconsistent
* situation during the compilation process.
*/
public OptimizedPlan compile(Plan program) throws CompilerException {
- // -------------------- try to get the connection to the job manager ----------------------
- // --------------------------to obtain instance information --------------------------------
final OptimizerPostPass postPasser = getPostPassFromPlan(program);
return compile(program, postPasser);
}
/**
- * Translates the given pact plan in to an OptimizedPlan, where all nodes have their local strategy assigned
- * and all channels have a shipping strategy assigned. The process goes through several phases:
+ * Translates the given program to an OptimizedPlan. The optimized plan describes for each operator
+ * which strategy to use (such as hash join versus sort-merge join), what data exchange method to use
+ * (local pipe forward, shuffle, broadcast), what exchange mode to use (pipelined, batch),
+ * where to cache intermediate results, etc,
+ *
+ * The optimization happens in multiple phases:
* <ol>
- * <li>Create <tt>OptimizerNode</tt> representations of the PACTs, assign parallelism and compute size estimates.</li>
+ * <li>Create optimizer dag implementation of the program.
+ *
+ * <tt>OptimizerNode</tt> representations of the PACTs, assign parallelism and compute size estimates.</li>
* <li>Compute interesting properties and auxiliary structures.</li>
* <li>Enumerate plan alternatives. This cannot be done in the same step as the interesting property computation (as
* opposed to the Database approaches), because we support plans that are not trees.</li>
http://git-wip-us.apache.org/repos/asf/flink/blob/7f8e1566/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
index c5ba395..41369c9 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
@@ -16,20 +16,24 @@
* limitations under the License.
*/
-
package org.apache.flink.compiler.dag;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.compiler.dataproperties.InterestingProperties;
import org.apache.flink.compiler.plandump.DumpableConnection;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
/**
- * A connection between to PACTs. Represents a channel together with a data shipping
- * strategy. The data shipping strategy of a channel may be fixed though compiler hints. In that
- * case, it is not modifiable.
- * <p>
- * The connections are also used by the optimization algorithm to propagate interesting properties from the sinks in the
- * direction of the sources.
+ * A connection between to operators. Represents an intermediate result
+ * and a data exchange between the two operators.
+ *
+ * The data exchange has a mode in which it performs (batch / pipelined)
+ *
+ * The data exchange strategy may be set on this connection, in which case
+ * it is fixed and will not be determined during candidate plan enumeration.
+ *
+ * During the enumeration of interesting properties, this connection also holds
+ * all interesting properties generated by the successor operator.
*/
public class PactConnection implements EstimateProvider, DumpableConnection<OptimizerNode> {
@@ -37,11 +41,13 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
private final OptimizerNode target; // The target node of the connection.
+ private final ExecutionMode dataExchangeMode; // defines whether to use batch or pipelined data exchange
+
private InterestingProperties interestingProps; // local properties that succeeding nodes are interested in
- private ShipStrategyType shipStrategy; // The data distribution strategy, if preset
+ private ShipStrategyType shipStrategy; // The data shipping strategy, if predefined.
- private TempMode materializationMode = TempMode.NONE;
+ private TempMode materializationMode = TempMode.NONE; // the materialization mode
private int maxDepth = -1;
@@ -54,8 +60,8 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
* @param target
* The target node.
*/
- public PactConnection(OptimizerNode source, OptimizerNode target) {
- this(source, target, null);
+ public PactConnection(OptimizerNode source, OptimizerNode target, ExecutionMode exchangeMode) {
+ this(source, target, null, exchangeMode);
}
/**
@@ -67,18 +73,24 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
* The target node.
* @param shipStrategy
* The shipping strategy.
+ * @param exchangeMode
+ * The data exchange mode (pipelined / batch)
*/
- public PactConnection(OptimizerNode source, OptimizerNode target, ShipStrategyType shipStrategy) {
+ public PactConnection(OptimizerNode source, OptimizerNode target,
+ ShipStrategyType shipStrategy, ExecutionMode exchangeMode)
+ {
if (source == null || target == null) {
throw new NullPointerException("Source and target must not be null.");
}
this.source = source;
this.target = target;
this.shipStrategy = shipStrategy;
+ this.dataExchangeMode = exchangeMode;
}
/**
- * Creates a new Connection between two nodes.
+ * Constructor to create a result from an operator that is not
+ * consumed by another operator.
*
* @param source
* The source node.
@@ -90,6 +102,7 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
this.source = source;
this.target = null;
this.shipStrategy = ShipStrategyType.NONE;
+ this.dataExchangeMode = null;
}
/**
@@ -130,6 +143,18 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
}
/**
+ * Gets the data exchange mode to use for this connection.
+ *
+ * @return The data exchange mode to use for this connection.
+ */
+ public ExecutionMode getDataExchangeMode() {
+ if (dataExchangeMode == null) {
+ throw new IllegalStateException("This connection does not have a data exchange");
+ }
+ return dataExchangeMode;
+ }
+
+ /**
* Gets the interesting properties object for this pact connection.
* If the interesting properties for this connections have not yet been set,
* this method returns null.
http://git-wip-us.apache.org/repos/asf/flink/blob/7f8e1566/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
----------------------------------------------------------------------
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 d315440..48a9c76 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
@@ -25,8 +25,24 @@ import java.util.ArrayList;
import java.util.List;
/**
- * A configuration config for configuring behavior of the system, such as whether to use
- * the closure cleaner, object-reuse mode...
+ * A config to define the behavior of the program execution. It allows to define (among other
+ * options) the following settings:
+ *
+ * <ul>
+ * <li>The default parallelism of the program, i.e., how many parallel tasks to use for
+ * all functions that do not define a specific value directly.</li>
+ * <li>The number of retries in the case of failed executions.</li>
+ * <li>The {@link ExecutionMode} of the program: Batch or Pipelined.
+ * The default execution mode is {@link ExecutionMode#PIPELINED}</li>
+ * <li>Enabling or disabling the "closure cleaner". The closure cleaner pre-processes
+ * the implementations of functions. In case they are (anonymous) inner classes,
+ * it removes unused references to the enclosing class to fix certain serialization-related
+ * problems and to reduce the size of the closure.</li>
+ * <li>The config allows to register types and serializers to increase the efficiency of
+ * handling <i>generic types</i> and <i>POJOs</i>. This is usually only needed
+ * when the functions return not only the types declared in their signature, but
+ * also subclasses of those types.</li>
+ * </ul>
*/
public class ExecutionConfig implements Serializable {
@@ -41,9 +57,17 @@ public class ExecutionConfig implements Serializable {
*/
public static final int PARALLELISM_AUTO_MAX = Integer.MAX_VALUE;
+ // --------------------------------------------------------------------------------------------
+
+ /** Defines how data exchange happens - batch or pipelined */
+ private ExecutionMode executionMode = ExecutionMode.PIPELINED;
+
private boolean useClosureCleaner = true;
- private int degreeOfParallelism = -1;
+
+ private int parallelism = -1;
+
private int numberOfExecutionRetries = -1;
+
private boolean forceKryo = false;
private boolean objectReuse = false;
@@ -52,17 +76,27 @@ public class ExecutionConfig implements Serializable {
private boolean serializeGenericTypesWithAvro = false;
-
-
// Serializers and types registered with Kryo and the PojoSerializer
// we store them in lists to ensure they are registered in order in all kryo instances.
- private final List<Entry<Class<?>, Serializer<?>>> registeredTypesWithKryoSerializers = new ArrayList<Entry<Class<?>, Serializer<?>>>();
- private final List<Entry<Class<?>, Class<? extends Serializer<?>>>> registeredTypesWithKryoSerializerClasses = new ArrayList<Entry<Class<?>, Class<? extends Serializer<?>>>>();
- private final List<Entry<Class<?>, Serializer<?>>> defaultKryoSerializers = new ArrayList<Entry<Class<?>, Serializer<?>>>();
- private final List<Entry<Class<?>, Class<? extends Serializer<?>>>> defaultKryoSerializerClasses = new ArrayList<Entry<Class<?>, Class<? extends Serializer<?>>>>();
+
+ private final List<Entry<Class<?>, Serializer<?>>> registeredTypesWithKryoSerializers =
+ new ArrayList<Entry<Class<?>, Serializer<?>>>();
+
+ private final List<Entry<Class<?>, Class<? extends Serializer<?>>>> registeredTypesWithKryoSerializerClasses =
+ new ArrayList<Entry<Class<?>, Class<? extends Serializer<?>>>>();
+
+ private final List<Entry<Class<?>, Serializer<?>>> defaultKryoSerializers =
+ new ArrayList<Entry<Class<?>, Serializer<?>>>();
+
+ private final List<Entry<Class<?>, Class<? extends Serializer<?>>>> defaultKryoSerializerClasses =
+ new ArrayList<Entry<Class<?>, Class<? extends Serializer<?>>>>();
+
private final List<Class<?>> registeredKryoTypes = new ArrayList<Class<?>>();
+
private final List<Class<?>> registeredPojoTypes = new ArrayList<Class<?>>();
+ // --------------------------------------------------------------------------------------------
+
/**
* Enables the ClosureCleaner. This analyzes user code functions and sets fields to null
* that are not used. This will in most cases make closures or anonymous inner classes
@@ -75,7 +109,9 @@ public class ExecutionConfig implements Serializable {
}
/**
- * Disables the ClosureCleaner. @see #enableClosureCleaner()
+ * Disables the ClosureCleaner.
+ *
+ * @see #enableClosureCleaner()
*/
public ExecutionConfig disableClosureCleaner() {
useClosureCleaner = false;
@@ -83,31 +119,32 @@ public class ExecutionConfig implements Serializable {
}
/**
- * Returns whether the ClosureCleaner is enabled. @see #enableClosureCleaner()
+ * Returns whether the ClosureCleaner is enabled.
+ *
+ * @see #enableClosureCleaner()
*/
public boolean isClosureCleanerEnabled() {
return useClosureCleaner;
}
/**
- * 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.
- * Other operations may need to run with a different
- * degree of parallelism - for example calling
- * a reduce operation over the entire
- * set will insert eventually an operation that runs non-parallel (degree of parallelism of one).
+ * Gets the parallelism with which operation are executed by default. Operations can
+ * individually override this value to use a specific parallelism.
*
- * @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.
+ * 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 getDegreeOfParallelism() {
- return degreeOfParallelism;
+ return parallelism;
}
/**
- * 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.
* <p>
* This method overrides the default parallelism for this environment.
@@ -115,14 +152,14 @@ public class ExecutionConfig implements Serializable {
* 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 to use
*/
-
- public ExecutionConfig setDegreeOfParallelism(int degreeOfParallelism) {
- if (degreeOfParallelism < 1) {
- throw new IllegalArgumentException("Degree of parallelism must be at least one.");
+ public ExecutionConfig setDegreeOfParallelism(int parallelism) {
+ if (parallelism < 1 && parallelism != -1) {
+ throw new IllegalArgumentException(
+ "Degree of parallelism must be at least one, or -1 (use system default).");
}
- this.degreeOfParallelism = degreeOfParallelism;
+ this.parallelism = parallelism;
return this;
}
@@ -146,13 +183,38 @@ public class ExecutionConfig implements Serializable {
*/
public ExecutionConfig setNumberOfExecutionRetries(int numberOfExecutionRetries) {
if (numberOfExecutionRetries < -1) {
- throw new IllegalArgumentException("The number of execution retries must be non-negative, or -1 (use system default)");
+ throw new IllegalArgumentException(
+ "The number of execution retries must be non-negative, or -1 (use system default)");
}
this.numberOfExecutionRetries = numberOfExecutionRetries;
return this;
}
/**
+ * Sets the execution mode to execute the program. The execution mode defines whether
+ * data exchanges are performed in a batch or on a pipelined manner.
+ *
+ * The default execution mode is {@link ExecutionMode#PIPELINED}.
+ *
+ * @param executionMode The execution mode to use.
+ */
+ public void setExecutionMode(ExecutionMode executionMode) {
+ this.executionMode = executionMode;
+ }
+
+ /**
+ * Gets the execution mode used to execute the program. The execution mode defines whether
+ * data exchanges are performed in a batch or on a pipelined manner.
+ *
+ * The default execution mode is {@link ExecutionMode#PIPELINED}.
+ *
+ * @return The execution mode for the program.
+ */
+ public ExecutionMode getExecutionMode() {
+ return executionMode;
+ }
+
+ /**
* Force TypeExtractor to use Kryo serializer for POJOS even though we could analyze as POJO.
* In some cases this might be preferable. For example, when using interfaces
* with subclasses that cannot be analyzed as POJO.
@@ -161,7 +223,6 @@ public class ExecutionConfig implements Serializable {
forceKryo = true;
}
-
/**
* Disable use of Kryo serializer for all POJOs.
*/
@@ -403,8 +464,12 @@ public class ExecutionConfig implements Serializable {
public static class Entry<K, V> implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
private final K k;
private final V v;
+
public Entry(K k, V v) {
this.k = k;
this.v = v;
http://git-wip-us.apache.org/repos/asf/flink/blob/7f8e1566/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java
new file mode 100644
index 0000000..f3e958e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common;
+
+/**
+ * The execution mode specifies how a batch program is executed in terms
+ * of data exchange: pipelining or batched.
+ */
+public enum ExecutionMode {
+
+ /**
+ * Executes the program in a pipelined fashion (including shuffles and broadcasts),
+ * except for data exchanges that are susceptible to deadlocks when pipelining.
+ * These data exchanges are performed in a batch manner.
+ *
+ * An example of situations that are susceptible to deadlocks (when executed in a
+ * pipelined manner) are data flows that branch (one data set consumed by multiple
+ * operations) and re-join later:
+ * <pre>{@code
+ * DataSet data = ...;
+ * DataSet mapped1 = data.map(new MyMapper());
+ * DataSet mapped2 = data.map(new AnotherMapper());
+ * mapped1.join(mapped2).where(...).equalTo(...);
+ * }</pre>
+ */
+ PIPELINED,
+
+ /**
+ * Executes the program in a pipelined fashion (including shuffles and broadcasts),
+ * <strong>including</strong> data exchanges that are susceptible to deadlocks when
+ * executed via pipelining.
+ *
+ * Usually, {@link #PIPELINED} is the preferable option, which pipelines most
+ * data exchanges and only uses batch data exchanges in situations that are
+ * susceptible to deadlocks.
+ *
+ * This option should only be used with care and only in situations where the
+ * programmer is sure that the program is safe for full pipelining and that
+ * Flink was too conservative when choosing the batch exchange at a certain
+ * point.
+ */
+ PIPELINED_FORCED,
+
+// This is for later, we are missing a bit of infrastructure for this.
+// /**
+// * The execution mode starts executing the program in a pipelined fashion
+// * (except for deadlock prone situations), similar to the {@link #PIPELINED}
+// * option. In the case of a task failure, re-execution happens in a batched
+// * mode, as defined for the {@link #BATCH} option.
+// */
+// PIPELINED_WITH_BATCH_FALLBACK,
+
+ /**
+ * This mode executes all shuffles and broadcasts in a batch fashion, while
+ * pipelining data between operations that exchange data only locally
+ * between one producer and one consumer.
+ */
+ BATCH,
+
+ /**
+ * This mode executes the program in a strict batch way, including all points
+ * where data is forwarded locally from one producer to one consumer. This mode
+ * is typically more expensive to execute than the {@link #BATCH} mode. It does
+ * guarantee that no successive operations are ever executed concurrently.
+ */
+ BATCH_FORCED
+}
[7/8] flink git commit: [FLINK-1671] [optimizer] Add data exchange
mode to optimizer classes
Posted by se...@apache.org.
[FLINK-1671] [optimizer] Add data exchange mode to optimizer classes
This closes #487
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/1c50d87c
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/1c50d87c
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/1c50d87c
Branch: refs/heads/master
Commit: 1c50d87c190f4845161bb3e281a5904efb32a605
Parents: 7f8e156
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Mar 11 23:35:11 2015 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Mar 17 10:10:08 2015 +0100
----------------------------------------------------------------------
.../org/apache/flink/compiler/PactCompiler.java | 116 ++++---
.../dag/AbstractPartialSolutionNode.java | 3 +-
.../flink/compiler/dag/BinaryUnionNode.java | 72 +++--
.../flink/compiler/dag/BulkIterationNode.java | 13 +-
.../apache/flink/compiler/dag/DataSinkNode.java | 28 +-
.../flink/compiler/dag/DataSourceNode.java | 19 +-
.../flink/compiler/dag/OptimizerNode.java | 122 ++++---
.../flink/compiler/dag/PactConnection.java | 32 +-
.../flink/compiler/dag/SingleInputNode.java | 61 ++--
.../apache/flink/compiler/dag/SinkJoiner.java | 7 +-
.../apache/flink/compiler/dag/TwoInputNode.java | 103 +++---
.../compiler/dag/WorksetIterationNode.java | 63 ++--
.../dataproperties/GlobalProperties.java | 64 +++-
.../RequestedGlobalProperties.java | 127 +++++---
.../AllGroupWithPartialPreGroupProperties.java | 16 +-
.../compiler/operators/AllReduceProperties.java | 22 +-
.../GroupReduceWithCombineProperties.java | 20 +-
.../compiler/operators/ReduceProperties.java | 18 +-
.../org/apache/flink/compiler/plan/Channel.java | 67 ++--
.../flink/compiler/plan/OptimizedPlan.java | 85 ++---
.../flink/compiler/AdditionalOperatorsTest.java | 3 -
.../apache/flink/compiler/CompilerTestBase.java | 112 +------
.../compiler/FeedbackPropertiesMatchTest.java | 113 +++----
.../costs/DefaultCostEstimatorTest.java | 3 -
.../DataExchangeModeClosedBranchingTest.java | 257 +++++++++++++++
.../DataExchangeModeForwardTest.java | 139 ++++++++
.../DataExchangeModeOpenBranchingTest.java | 182 +++++++++++
.../dataexchange/PipelineBreakingTest.java | 320 +++++++++++++++++++
.../testfunctions/IdentityFlatMapper.java | 30 ++
.../flink/api/common/operators/Ordering.java | 13 +-
.../runtime/io/network/DataExchangeMode.java | 128 ++++++++
.../io/network/DataExchangeModeTest.java | 51 +++
32 files changed, 1782 insertions(+), 627 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java b/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
index d5cbf68..0ea8724 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/PactCompiler.java
@@ -28,10 +28,12 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
-import org.apache.flink.api.common.operators.base.SortPartitionOperatorBase;
-import org.apache.flink.compiler.dag.SortPartitionNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.apache.flink.api.common.operators.base.SortPartitionOperatorBase;
+import org.apache.flink.compiler.dag.SortPartitionNode;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.operators.GenericDataSinkBase;
@@ -487,14 +489,14 @@ public class PactCompiler {
LOG.debug("Beginning compilation of program '" + program.getJobName() + '\'');
}
- // set the default degree of parallelism
- int defaultParallelism = program.getDefaultParallelism() > 0 ?
+ final ExecutionMode defaultDataExchangeMode = program.getExecutionConfig().getExecutionMode();
+
+ final int defaultParallelism = program.getDefaultParallelism() > 0 ?
program.getDefaultParallelism() : this.defaultDegreeOfParallelism;
- // log the output
- if (LOG.isDebugEnabled()) {
- LOG.debug("Using a default degree of parallelism of " + defaultParallelism + '.');
- }
+ // log the default settings
+ LOG.debug("Using a default parallelism of {}", defaultParallelism);
+ LOG.debug("Using default data exchange mode {}", defaultDataExchangeMode);
// the first step in the compilation is to create the optimizer plan representation
// this step does the following:
@@ -505,7 +507,7 @@ public class PactCompiler {
// 4) It makes estimates about the data volume of the data sources and
// propagates those estimates through the plan
- GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(defaultParallelism);
+ GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(defaultParallelism, defaultDataExchangeMode);
program.accept(graphCreator);
// if we have a plan with multiple data sinks, add logical optimizer nodes that have two data-sinks as children
@@ -527,21 +529,17 @@ public class PactCompiler {
// now that we have all nodes created and recorded which ones consume memory, tell the nodes their minimal
// guaranteed memory, for further cost estimations. we assume an equal distribution of memory among consumer tasks
-
rootNode.accept(new IdAndEstimatesVisitor(this.statistics));
-
- // Now that the previous step is done, the next step is to traverse the graph again for the two
- // steps that cannot directly be performed during the plan enumeration, because we are dealing with DAGs
- // rather than a trees. That requires us to deviate at some points from the classical DB optimizer algorithms.
- //
- // 1) propagate the interesting properties top-down through the graph
- // 2) Track information about nodes with multiple outputs that are later on reconnected in a node with
- // multiple inputs.
- InterestingPropertyVisitor propsVisitor = new InterestingPropertyVisitor(this.costEstimator);
- rootNode.accept(propsVisitor);
-
+
+ // We are dealing with operator DAGs, rather than operator trees.
+ // That requires us to deviate at some points from the classical DB optimizer algorithms.
+ // This step build some auxiliary structures to help track branches and joins in the DAG
BranchesVisitor branchingVisitor = new BranchesVisitor();
rootNode.accept(branchingVisitor);
+
+ // Propagate the interesting properties top-down through the graph
+ InterestingPropertyVisitor propsVisitor = new InterestingPropertyVisitor(this.costEstimator);
+ rootNode.accept(propsVisitor);
// perform a sanity check: the root may not have any unclosed branches
if (rootNode.getOpenBranches() != null && rootNode.getOpenBranches().size() > 0) {
@@ -590,7 +588,7 @@ public class PactCompiler {
* from the plan can be traversed.
*/
public static List<DataSinkNode> createPreOptimizedPlan(Plan program) {
- GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(1);
+ GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(1, null);
program.accept(graphCreator);
return graphCreator.sinks;
}
@@ -609,40 +607,45 @@ public class PactCompiler {
* estimation and the awareness for optimizer hints, the sizes will be properly estimated and the translated plan
* already respects all optimizer hints.
*/
- private static final class GraphCreatingVisitor implements Visitor<Operator<?>> {
+ public static final class GraphCreatingVisitor implements Visitor<Operator<?>> {
private final Map<Operator<?>, OptimizerNode> con2node; // map from the operator objects to their
// corresponding optimizer nodes
- private final List<DataSourceNode> sources; // all data source nodes in the optimizer plan
-
private final List<DataSinkNode> sinks; // all data sink nodes in the optimizer plan
private final int defaultParallelism; // the default degree of parallelism
private final GraphCreatingVisitor parent; // reference to enclosing creator, in case of a recursive translation
-
+
+ private final ExecutionMode defaultDataExchangeMode;
+
private final boolean forceDOP;
- private GraphCreatingVisitor(int defaultParallelism) {
- this(null, false, defaultParallelism, null);
+ public GraphCreatingVisitor(int defaultParallelism, ExecutionMode defaultDataExchangeMode) {
+ this(null, false, defaultParallelism, defaultDataExchangeMode, null);
}
- private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceDOP,
- int defaultParallelism, HashMap<Operator<?>, OptimizerNode> closure) {
+ private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceDOP, int defaultParallelism,
+ ExecutionMode dataExchangeMode, HashMap<Operator<?>, OptimizerNode> closure) {
if (closure == null){
con2node = new HashMap<Operator<?>, OptimizerNode>();
} else {
con2node = closure;
}
- this.sources = new ArrayList<DataSourceNode>(4);
+
this.sinks = new ArrayList<DataSinkNode>(2);
this.defaultParallelism = defaultParallelism;
this.parent = parent;
+ this.defaultDataExchangeMode = dataExchangeMode;
this.forceDOP = forceDOP;
}
+ public List<DataSinkNode> getSinks() {
+ return sinks;
+ }
+
@SuppressWarnings("deprecation")
@Override
public boolean preVisit(Operator<?> c) {
@@ -660,9 +663,7 @@ public class PactCompiler {
n = dsn;
}
else if (c instanceof GenericDataSourceBase) {
- DataSourceNode dsn = new DataSourceNode((GenericDataSourceBase<?, ?>) c);
- this.sources.add(dsn);
- n = dsn;
+ n = new DataSourceNode((GenericDataSourceBase<?, ?>) c);
}
else if (c instanceof MapOperatorBase) {
n = new MapNode((MapOperatorBase<?, ?, ?>) c);
@@ -768,8 +769,8 @@ public class PactCompiler {
if (par > 0) {
if (this.forceDOP && par != this.defaultParallelism) {
par = this.defaultParallelism;
- LOG.warn("The degree-of-parallelism of nested Dataflows (such as step functions in iterations) is " +
- "currently fixed to the degree-of-parallelism of the surrounding operator (the iteration).");
+ 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).");
}
} else {
par = this.defaultParallelism;
@@ -786,8 +787,8 @@ public class PactCompiler {
OptimizerNode n = this.con2node.get(c);
// first connect to the predecessors
- n.setInput(this.con2node);
- n.setBroadcastInputs(this.con2node);
+ n.setInput(this.con2node, this.defaultDataExchangeMode);
+ n.setBroadcastInputs(this.con2node, this.defaultDataExchangeMode);
// if the node represents a bulk iteration, we recursively translate the data flow now
if (n instanceof BulkIterationNode) {
@@ -800,9 +801,9 @@ public class PactCompiler {
// first, recursively build the data flow for the step function
final GraphCreatingVisitor recursiveCreator = new GraphCreatingVisitor(this, true,
- iterNode.getDegreeOfParallelism(), closure);
+ iterNode.getDegreeOfParallelism(), defaultDataExchangeMode, closure);
- BulkPartialSolutionNode partialSolution = null;
+ BulkPartialSolutionNode partialSolution;
iter.getNextPartialSolution().accept(recursiveCreator);
@@ -836,13 +837,14 @@ public class PactCompiler {
final WorksetIterationNode iterNode = (WorksetIterationNode) n;
final DeltaIterationBase<?, ?> iter = iterNode.getIterationContract();
- // we need to ensure that both the next-workset and the solution-set-delta depend on the workset. One check is for free
- // during the translation, we do the other check here as a pre-condition
+ // we need to ensure that both the next-workset and the solution-set-delta depend on the workset.
+ // One check is for free during the translation, we do the other check here as a pre-condition
{
StepFunctionValidator wsf = new StepFunctionValidator();
iter.getNextWorkset().accept(wsf);
if (!wsf.foundWorkset) {
- throw new CompilerException("In the given program, the next workset does not depend on the workset. This is a prerequisite in delta iterations.");
+ throw new CompilerException("In the given program, the next workset does not depend on the workset. " +
+ "This is a prerequisite in delta iterations.");
}
}
@@ -850,7 +852,8 @@ public class PactCompiler {
HashMap<Operator<?>, OptimizerNode> closure = new HashMap<Operator<?>, OptimizerNode>(con2node);
// first, recursively build the data flow for the step function
- final GraphCreatingVisitor recursiveCreator = new GraphCreatingVisitor(this, true, iterNode.getDegreeOfParallelism(), closure);
+ final GraphCreatingVisitor recursiveCreator = new GraphCreatingVisitor(
+ this, true, iterNode.getDegreeOfParallelism(), defaultDataExchangeMode, closure);
// descend from the solution set delta. check that it depends on both the workset
// and the solution set. If it does depend on both, this descend should create both nodes
@@ -859,7 +862,8 @@ public class PactCompiler {
final WorksetNode worksetNode = (WorksetNode) recursiveCreator.con2node.get(iter.getWorkset());
if (worksetNode == null) {
- throw new CompilerException("In the given program, the solution set delta does not depend on the workset. This is a prerequisite in delta iterations.");
+ throw new CompilerException("In the given program, the solution set delta does not depend on the workset." +
+ "This is a prerequisite in delta iterations.");
}
iter.getNextWorkset().accept(recursiveCreator);
@@ -895,7 +899,8 @@ public class PactCompiler {
}
}
else {
- throw new InvalidProgramException("Error: The only operations allowed on the solution set are Join and CoGroup.");
+ throw new InvalidProgramException(
+ "Error: The only operations allowed on the solution set are Join and CoGroup.");
}
}
}
@@ -905,14 +910,14 @@ public class PactCompiler {
// set the step function nodes to the iteration node
iterNode.setPartialSolution(solutionSetNode, worksetNode);
- iterNode.setNextPartialSolution(solutionSetDeltaNode, nextWorksetNode);
+ iterNode.setNextPartialSolution(solutionSetDeltaNode, nextWorksetNode, defaultDataExchangeMode);
// go over the contained data flow and mark the dynamic path nodes
StaticDynamicPathIdentifier pathIdentifier = new StaticDynamicPathIdentifier(iterNode.getCostWeight());
iterNode.acceptForStepFunction(pathIdentifier);
}
}
- };
+ }
private static final class StaticDynamicPathIdentifier implements Visitor<OptimizerNode> {
@@ -944,28 +949,21 @@ public class PactCompiler {
* Simple visitor that sets the minimal guaranteed memory per task based on the amount of available memory,
* the number of memory consumers, and on the task's degree of parallelism.
*/
- private static final class IdAndEstimatesVisitor implements Visitor<OptimizerNode> {
+ public static final class IdAndEstimatesVisitor implements Visitor<OptimizerNode> {
private final DataStatistics statistics;
private int id = 1;
- private IdAndEstimatesVisitor(DataStatistics statistics) {
+ public IdAndEstimatesVisitor(DataStatistics statistics) {
this.statistics = statistics;
}
-
@Override
public boolean preVisit(OptimizerNode visitable) {
- if (visitable.getId() != -1) {
- // been here before
- return false;
- }
-
- return true;
+ return visitable.getId() == -1;
}
-
@Override
public void postVisit(OptimizerNode visitable) {
// the node ids
@@ -1031,7 +1029,7 @@ public class PactCompiler {
* that are not a minimally connected DAG (Such plans are not trees, but at least one node feeds its
* output into more than one other node).
*/
- private static final class BranchesVisitor implements Visitor<OptimizerNode> {
+ public static final class BranchesVisitor implements Visitor<OptimizerNode> {
@Override
public boolean preVisit(OptimizerNode node) {
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/AbstractPartialSolutionNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/AbstractPartialSolutionNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/AbstractPartialSolutionNode.java
index d996fe9..957dd1a 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/AbstractPartialSolutionNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/AbstractPartialSolutionNode.java
@@ -23,6 +23,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.operators.Operator;
import org.apache.flink.api.common.operators.SemanticProperties;
import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties;
@@ -66,7 +67,7 @@ public abstract class AbstractPartialSolutionNode extends OptimizerNode {
}
@Override
- public void setInput(Map<Operator<?>, OptimizerNode> contractToNode) {}
+ public void setInput(Map<Operator<?>, OptimizerNode> contractToNode, ExecutionMode dataExchangeMode) {}
@Override
protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) {
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BinaryUnionNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BinaryUnionNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BinaryUnionNode.java
index ddc776e..b0db623 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BinaryUnionNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BinaryUnionNode.java
@@ -20,10 +20,10 @@ package org.apache.flink.compiler.dag;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashSet;
import java.util.List;
import java.util.Set;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.operators.SemanticProperties;
import org.apache.flink.api.common.operators.Union;
import org.apache.flink.api.common.operators.util.FieldSet;
@@ -39,6 +39,7 @@ import org.apache.flink.compiler.operators.OperatorDescriptorDual;
import org.apache.flink.compiler.plan.Channel;
import org.apache.flink.compiler.plan.NamedChannel;
import org.apache.flink.compiler.plan.PlanNode;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
/**
@@ -105,30 +106,22 @@ public class BinaryUnionNode extends TwoInputNode {
// step down to all producer nodes and calculate alternative plans
final List<? extends PlanNode> subPlans1 = getFirstPredecessorNode().getAlternativePlans(estimator);
final List<? extends PlanNode> subPlans2 = getSecondPredecessorNode().getAlternativePlans(estimator);
-
- // calculate alternative sub-plans for broadcast inputs
- final List<Set<? extends NamedChannel>> broadcastPlanChannels = new ArrayList<Set<? extends NamedChannel>>();
+
List<PactConnection> broadcastConnections = getBroadcastConnections();
- List<String> broadcastConnectionNames = getBroadcastConnectionNames();
- for (int i = 0; i < broadcastConnections.size(); i++ ) {
- PactConnection broadcastConnection = broadcastConnections.get(i);
- String broadcastConnectionName = broadcastConnectionNames.get(i);
- List<PlanNode> broadcastPlanCandidates = broadcastConnection.getSource().getAlternativePlans(estimator);
- // wrap the plan candidates in named channels
- HashSet<NamedChannel> broadcastChannels = new HashSet<NamedChannel>(broadcastPlanCandidates.size());
- for (PlanNode plan: broadcastPlanCandidates) {
- final NamedChannel c = new NamedChannel(broadcastConnectionName, plan);
- c.setShipStrategy(ShipStrategyType.BROADCAST);
- broadcastChannels.add(c);
- }
- broadcastPlanChannels.add(broadcastChannels);
+ if (broadcastConnections != null && broadcastConnections.size() > 0) {
+ throw new CompilerException("Found BroadcastVariables on a Union operation");
}
final ArrayList<PlanNode> outputPlans = new ArrayList<PlanNode>();
-
+
+ final List<Set<? extends NamedChannel>> broadcastPlanChannels = Collections.emptyList();
+
final BinaryUnionOpDescriptor operator = new BinaryUnionOpDescriptor();
final RequestedLocalProperties noLocalProps = new RequestedLocalProperties();
-
+
+ final ExecutionMode input1Mode = this.input1.getDataExchangeMode();
+ final ExecutionMode input2Mode = this.input2.getDataExchangeMode();
+
final int dop = getDegreeOfParallelism();
final int inDop1 = getFirstPredecessorNode().getDegreeOfParallelism();
final int inDop2 = getSecondPredecessorNode().getDegreeOfParallelism();
@@ -136,6 +129,9 @@ public class BinaryUnionNode extends TwoInputNode {
final boolean dopChange1 = dop != inDop1;
final boolean dopChange2 = dop != inDop2;
+ final boolean input1breakPipeline = this.input1.isBreakingPipeline();
+ final boolean input2breakPipeline = this.input2.isBreakingPipeline();
+
// enumerate all pairwise combination of the children's plans together with
// all possible operator strategy combination
@@ -154,19 +150,22 @@ public class BinaryUnionNode extends TwoInputNode {
Channel c1 = new Channel(child1, this.input1.getMaterializationMode());
if (this.input1.getShipStrategy() == null) {
// free to choose the ship strategy
- igps.parameterizeChannel(c1, dopChange1);
+ 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 (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
c1.getGlobalProperties().reset();
}
- } else {
+ }
+ else {
// ship strategy fixed by compiler hint
+ ShipStrategyType shipStrategy = this.input1.getShipStrategy();
+ DataExchangeMode exMode = DataExchangeMode.select(input1Mode, shipStrategy, input1breakPipeline);
if (this.keys1 != null) {
- c1.setShipStrategy(this.input1.getShipStrategy(), this.keys1.toFieldList());
+ c1.setShipStrategy(this.input1.getShipStrategy(), this.keys1.toFieldList(), exMode);
} else {
- c1.setShipStrategy(this.input1.getShipStrategy());
+ c1.setShipStrategy(this.input1.getShipStrategy(), exMode);
}
if (dopChange1) {
@@ -178,7 +177,7 @@ public class BinaryUnionNode extends TwoInputNode {
Channel c2 = new Channel(child2, this.input2.getMaterializationMode());
if (this.input2.getShipStrategy() == null) {
// free to choose the ship strategy
- igps.parameterizeChannel(c2, dopChange2);
+ 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
@@ -187,10 +186,12 @@ public class BinaryUnionNode extends TwoInputNode {
}
} else {
// ship strategy fixed by compiler hint
+ ShipStrategyType shipStrategy = this.input2.getShipStrategy();
+ DataExchangeMode exMode = DataExchangeMode.select(input2Mode, shipStrategy, input2breakPipeline);
if (this.keys2 != null) {
- c2.setShipStrategy(this.input2.getShipStrategy(), this.keys2.toFieldList());
+ c2.setShipStrategy(this.input2.getShipStrategy(), this.keys2.toFieldList(), exMode);
} else {
- c2.setShipStrategy(this.input2.getShipStrategy());
+ c2.setShipStrategy(this.input2.getShipStrategy(), exMode);
}
if (dopChange2) {
@@ -204,7 +205,7 @@ public class BinaryUnionNode extends TwoInputNode {
p1.clearUniqueFieldCombinations();
p2.clearUniqueFieldCombinations();
- // adjust the partitionings, if they exist but are not equal. this may happen when both channels have a
+ // adjust the partitioning, if they exist but are not equal. this may happen when both channels have a
// partitioning that fulfills the requirements, but both are incompatible. For example may a property requirement
// be ANY_PARTITIONING on fields (0) and one channel is range partitioned on that field, the other is hash
// partitioned on that field.
@@ -212,20 +213,22 @@ public class BinaryUnionNode extends TwoInputNode {
if (c1.getShipStrategy() == ShipStrategyType.FORWARD && c2.getShipStrategy() != ShipStrategyType.FORWARD) {
// adjust c2 to c1
c2 = c2.clone();
- p1.parameterizeChannel(c2,dopChange2);
- } else if (c2.getShipStrategy() == ShipStrategyType.FORWARD && c1.getShipStrategy() != ShipStrategyType.FORWARD) {
+ p1.parameterizeChannel(c2, dopChange2, input2Mode, input2breakPipeline);
+ }
+ else if (c2.getShipStrategy() == ShipStrategyType.FORWARD && c1.getShipStrategy() != ShipStrategyType.FORWARD) {
// adjust c1 to c2
c1 = c1.clone();
- p2.parameterizeChannel(c1,dopChange1);
- } else if (c1.getShipStrategy() == ShipStrategyType.FORWARD && c2.getShipStrategy() == ShipStrategyType.FORWARD) {
+ p2.parameterizeChannel(c1,dopChange1, input1Mode, input1breakPipeline);
+ }
+ else if (c1.getShipStrategy() == ShipStrategyType.FORWARD && c2.getShipStrategy() == ShipStrategyType.FORWARD) {
boolean adjustC1 = c1.getEstimatedOutputSize() <= 0 || c2.getEstimatedOutputSize() <= 0 ||
c1.getEstimatedOutputSize() <= c2.getEstimatedOutputSize();
if (adjustC1) {
c2 = c2.clone();
- p1.parameterizeChannel(c2, dopChange2);
+ p1.parameterizeChannel(c2, dopChange2, input2Mode, input2breakPipeline);
} else {
c1 = c1.clone();
- p2.parameterizeChannel(c1, dopChange1);
+ p2.parameterizeChannel(c1, dopChange1, input1Mode, input1breakPipeline);
}
} else {
// this should never happen, as it implies both realize a different strategy, which is
@@ -233,7 +236,8 @@ public class BinaryUnionNode extends TwoInputNode {
throw new CompilerException("Bug in Plan Enumeration for Union Node.");
}
}
-
+
+
instantiate(operator, c1, c2, broadcastPlanChannels, outputPlans, estimator, igps, igps, noLocalProps, noLocalProps);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BulkIterationNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BulkIterationNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BulkIterationNode.java
index 43b5799..ac94ef3 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BulkIterationNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/BulkIterationNode.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Set;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.operators.SemanticProperties;
import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties;
import org.apache.flink.api.common.operators.base.BulkIterationBase;
@@ -139,7 +140,7 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
NoOpNode noop = new NoOpNode();
noop.setDegreeOfParallelism(getDegreeOfParallelism());
- PactConnection noOpConn = new PactConnection(nextPartialSolution, noop);
+ PactConnection noOpConn = new PactConnection(nextPartialSolution, noop, ExecutionMode.PIPELINED);
noop.setIncomingConnection(noOpConn);
nextPartialSolution.addOutgoingConnection(noOpConn);
@@ -151,13 +152,15 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
if (terminationCriterion == null) {
this.singleRoot = nextPartialSolution;
- this.rootConnection = new PactConnection(nextPartialSolution);
+ this.rootConnection = new PactConnection(nextPartialSolution, ExecutionMode.PIPELINED);
}
else {
// we have a termination criterion
SingleRootJoiner singleRootJoiner = new SingleRootJoiner();
- this.rootConnection = new PactConnection(nextPartialSolution, singleRootJoiner);
- this.terminationCriterionRootConnection = new PactConnection(terminationCriterion, singleRootJoiner);
+ this.rootConnection = new PactConnection(nextPartialSolution, singleRootJoiner, ExecutionMode.PIPELINED);
+ this.terminationCriterionRootConnection = new PactConnection(terminationCriterion, singleRootJoiner,
+ ExecutionMode.PIPELINED);
+
singleRootJoiner.setInputs(this.rootConnection, this.terminationCriterionRootConnection);
this.singleRoot = singleRootJoiner;
@@ -316,7 +319,7 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) {
// attach a no-op node through which we create the properties of the original input
Channel toNoOp = new Channel(candidate);
- globPropsReq.parameterizeChannel(toNoOp, false);
+ globPropsReq.parameterizeChannel(toNoOp, false, rootConnection.getDataExchangeMode(), false);
locPropsReq.parameterizeChannel(toNoOp);
UnaryOperatorNode rebuildPropertiesNode = new UnaryOperatorNode("Rebuild Partial Solution Properties", FieldList.EMPTY_LIST);
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSinkNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSinkNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSinkNode.java
index aa80451..2a0ef2b 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSinkNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSinkNode.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.compiler.dag;
import java.util.ArrayList;
@@ -24,6 +23,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.distributions.DataDistribution;
import org.apache.flink.api.common.operators.GenericDataSinkBase;
import org.apache.flink.api.common.operators.Operator;
@@ -69,7 +69,9 @@ public class DataSinkNode extends OptimizerNode {
}
/**
- *
+ * Gets the predecessor of this node.
+ *
+ * @return The predecessor, or null, if no predecessor has been set.
*/
public OptimizerNode getPredecessorNode() {
if(this.input != null) {
@@ -80,9 +82,9 @@ public class DataSinkNode extends OptimizerNode {
}
/**
- * Gets the contract object for this data source node.
+ * Gets the operator for which this optimizer sink node was created.
*
- * @return The contract.
+ * @return The node's underlying operator.
*/
@Override
public GenericDataSinkBase<?> getPactContract() {
@@ -99,19 +101,25 @@ public class DataSinkNode extends OptimizerNode {
return Collections.singletonList(this.input);
}
+ /**
+ * Gets all outgoing connections, which is an empty set for the data sink.
+ *
+ * @return An empty list.
+ */
+ @Override
public List<PactConnection> getOutgoingConnections() {
return Collections.emptyList();
}
@Override
- public void setInput(Map<Operator<?>, OptimizerNode> contractToNode) {
+ public void setInput(Map<Operator<?>, OptimizerNode> contractToNode, ExecutionMode defaultExchangeMode) {
Operator<?> children = getPactContract().getInput();
final OptimizerNode pred;
final PactConnection conn;
pred = contractToNode.get(children);
- conn = new PactConnection(pred, this);
+ conn = new PactConnection(pred, this, defaultExchangeMode);
// create the connection and add it
this.input = conn;
@@ -170,7 +178,7 @@ public class DataSinkNode extends OptimizerNode {
}
// we need to track open branches even in the sinks, because they get "closed" when
- // we build a single "roor" for the data flow plan
+ // we build a single "root" for the data flow plan
addClosedBranches(getPredecessorNode().closedBranchingNodes);
this.openBranches = getPredecessorNode().getBranchesForParent(this.input);
}
@@ -199,14 +207,16 @@ public class DataSinkNode extends OptimizerNode {
final int dop = getDegreeOfParallelism();
final int inDop = getPredecessorNode().getDegreeOfParallelism();
+ final ExecutionMode executionMode = this.input.getDataExchangeMode();
final boolean dopChange = dop != inDop;
+ final boolean breakPipeline = this.input.isBreakingPipeline();
InterestingProperties ips = this.input.getInterestingProperties();
for (PlanNode p : subPlans) {
for (RequestedGlobalProperties gp : ips.getGlobalProperties()) {
for (RequestedLocalProperties lp : ips.getLocalProperties()) {
Channel c = new Channel(p);
- gp.parameterizeChannel(c, dopChange);
+ gp.parameterizeChannel(c, dopChange, executionMode, breakPipeline);
lp.parameterizeChannel(c);
c.setRequiredLocalProps(lp);
c.setRequiredGlobalProps(gp);
@@ -214,7 +224,7 @@ public class DataSinkNode extends OptimizerNode {
// no need to check whether the created properties meet what we need in case
// of ordering or global ordering, because the only interesting properties we have
// are what we require
- outputPlans.add(new SinkPlanNode(this, "DataSink("+this.getPactContract().getName()+")" ,c));
+ outputPlans.add(new SinkPlanNode(this, "DataSink ("+this.getPactContract().getName()+")" ,c));
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSourceNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSourceNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSourceNode.java
index 49946e0..3454db0 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSourceNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/DataSourceNode.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.compiler.dag;
import java.util.ArrayList;
@@ -24,6 +23,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.common.io.FileInputFormat;
import org.apache.flink.api.common.io.InputFormat;
@@ -81,11 +81,8 @@ public class DataSourceNode extends OptimizerNode {
this.sequentialInput = false;
}
- if (pactContract.getUserCodeWrapper().getUserCodeObject() instanceof ReplicatingInputFormat) {
- this.replicatedInput = true;
- } else {
- this.replicatedInput = false;
- }
+ this.replicatedInput = ReplicatingInputFormat.class.isAssignableFrom(
+ pactContract.getUserCodeWrapper().getUserCodeClass());
this.gprops = new GlobalProperties();
this.lprops = new LocalProperties();
@@ -119,7 +116,7 @@ public class DataSourceNode extends OptimizerNode {
@Override
public void setDegreeOfParallelism(int degreeOfParallelism) {
- // if unsplittable, DOP remains at 1
+ // if unsplittable, parallelism remains at 1
if (!this.sequentialInput) {
super.setDegreeOfParallelism(degreeOfParallelism);
}
@@ -131,14 +128,14 @@ public class DataSourceNode extends OptimizerNode {
}
@Override
- public void setInput(Map<Operator<?>, OptimizerNode> contractToNode) {}
+ public void setInput(Map<Operator<?>, OptimizerNode> contractToNode, ExecutionMode defaultDataExchangeMode) {}
@Override
protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) {
// see, if we have a statistics object that can tell us a bit about the file
if (statistics != null) {
// instantiate the input format, as this is needed by the statistics
- InputFormat<?, ?> format = null;
+ InputFormat<?, ?> format;
String inFormatDescription = "<unknown>";
try {
@@ -156,7 +153,9 @@ public class DataSourceNode extends OptimizerNode {
try {
inFormatDescription = format.toString();
}
- catch (Throwable t) {}
+ catch (Throwable t) {
+ // we can ignore this error, as it only prevents us to use a cosmetic string
+ }
// first of all, get the statistics from the cache
final String statisticsKey = getPactContract().getStatisticsKey();
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
index 4e4ef28..16c60a4 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/OptimizerNode.java
@@ -28,6 +28,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.operators.AbstractUdfOperator;
import org.apache.flink.api.common.operators.CompilerHints;
import org.apache.flink.api.common.operators.Operator;
@@ -133,42 +134,39 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
}
// ------------------------------------------------------------------------
- // Abstract methods that implement node specific behavior
- // and the pact type specific optimization methods.
+ // Methods specific to unary- / binary- / special nodes
// ------------------------------------------------------------------------
/**
- * Gets the name of this node. This returns either the name of the PACT, or
- * a string marking the node as a data source or a data sink.
+ * Gets the name of this node, which is the name of the function/operator, or
+ * data source / data sink.
*
* @return The node name.
*/
public abstract String getName();
/**
- * This function is for plan translation purposes. Upon invocation, the implementing subclasses should
- * examine its contained contract and look at the contracts that feed their data into that contract.
- * The method should then create a <tt>PactConnection</tt> for each of those inputs.
- * <p>
- * In addition, the nodes must set the shipping strategy of the connection, if a suitable optimizer hint is found.
- *
- * @param contractToNode
- * The map to translate the contracts to their corresponding optimizer nodes.
+ * This function connects the predecessors to this operator.
+ *
+ * @param operatorToNode The map from program operators to optimizer nodes.
+ * @param defaultExchangeMode The data exchange mode to use, if the operator does not
+ * specify one.
*/
- public abstract void setInput(Map<Operator<?>, OptimizerNode> contractToNode);
+ public abstract void setInput(Map<Operator<?>, OptimizerNode> operatorToNode,
+ ExecutionMode defaultExchangeMode);
/**
- * This function is for plan translation purposes. Upon invocation, this method creates a {@link PactConnection}
- * for each one of the broadcast inputs associated with the {@code Operator} referenced by this node.
- * <p>
- * The {@code PactConnections} must set its shipping strategy type to BROADCAST.
- *
- * @param operatorToNode
- * The map associating operators with their corresponding optimizer nodes.
+ * This function connects the predecessors to this operator.
+ *
+ * @param operatorToNode The map from program operators to optimizer nodes.
+ * @param defaultExchangeMode The data exchange mode to use, if the operator does not
+ * specify one.
+ *
* @throws CompilerException
*/
- public void setBroadcastInputs(Map<Operator<?>, OptimizerNode> operatorToNode) throws CompilerException {
-
+ public void setBroadcastInputs(Map<Operator<?>, OptimizerNode> operatorToNode, ExecutionMode defaultExchangeMode)
+ throws CompilerException
+ {
// skip for Operators that don't support broadcast variables
if (!(getPactContract() instanceof AbstractUdfOperator<?, ?>)) {
return;
@@ -180,7 +178,8 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
// create connections and add them
for (Map.Entry<String, Operator<?>> input : operator.getBroadcastInputs().entrySet()) {
OptimizerNode predecessor = operatorToNode.get(input.getValue());
- PactConnection connection = new PactConnection(predecessor, this, ShipStrategyType.BROADCAST);
+ PactConnection connection = new PactConnection(predecessor, this,
+ ShipStrategyType.BROADCAST, defaultExchangeMode);
addBroadcastConnection(input.getKey(), connection);
predecessor.addOutgoingConnection(connection);
}
@@ -189,7 +188,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
/**
* This method needs to be overridden by subclasses to return the children.
*
- * @return The list of incoming links.
+ * @return The list of incoming connections.
*/
public abstract List<PactConnection> getIncomingConnections();
@@ -198,7 +197,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
* for the node itself must have been computed before.
* The node must then see how many of interesting properties it preserves and add its own.
*
- * @param estimator The {@code CostEstimator} instance to use for plan cost estimation.
+ * @param estimator The {@code CostEstimator} instance to use for plan cost estimation.
*/
public abstract void computeInterestingPropertiesForInputs(CostEstimator estimator);
@@ -212,7 +211,9 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
public abstract void computeUnclosedBranchStack();
- protected List<UnclosedBranchDescriptor> computeUnclosedBranchStackForBroadcastInputs(List<UnclosedBranchDescriptor> branchesSoFar) {
+ protected List<UnclosedBranchDescriptor> computeUnclosedBranchStackForBroadcastInputs(
+ List<UnclosedBranchDescriptor> branchesSoFar)
+ {
// handle the data flow branching for the broadcast inputs
for (PactConnection broadcastInput : getBroadcastConnections()) {
OptimizerNode bcSource = broadcastInput.getSource();
@@ -221,7 +222,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
List<UnclosedBranchDescriptor> bcBranches = bcSource.getBranchesForParent(broadcastInput);
ArrayList<UnclosedBranchDescriptor> mergedBranches = new ArrayList<UnclosedBranchDescriptor>();
- mergeLists(branchesSoFar, bcBranches, mergedBranches);
+ mergeLists(branchesSoFar, bcBranches, mergedBranches, true);
branchesSoFar = mergedBranches.isEmpty() ? Collections.<UnclosedBranchDescriptor>emptyList() : mergedBranches;
}
@@ -261,9 +262,9 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
@Override
public Iterable<OptimizerNode> getPredecessors() {
List<OptimizerNode> allPredecessors = new ArrayList<OptimizerNode>();
-
- for (Iterator<PactConnection> inputs = getIncomingConnections().iterator(); inputs.hasNext(); ){
- allPredecessors.add(inputs.next().getSource());
+
+ for (PactConnection pactConnection : getIncomingConnections()) {
+ allPredecessors.add(pactConnection.getSource());
}
for (PactConnection conn : getBroadcastConnections()) {
@@ -303,8 +304,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
/**
* Adds the broadcast connection identified by the given {@code name} to this node.
*
- * @param broadcastConnection
- * The connection to add.
+ * @param broadcastConnection The connection to add.
*/
public void addBroadcastConnection(String name, PactConnection broadcastConnection) {
this.broadcastConnectionNames.add(name);
@@ -353,42 +353,40 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
}
/**
- * Gets the object that specifically describes the contract of this node.
+ * Gets the operator represented by this optimizer node.
*
- * @return This node's contract.
+ * @return This node's operator.
*/
public Operator<?> getPactContract() {
return this.pactContract;
}
/**
- * Gets the degree of parallelism for the contract represented by this optimizer node.
- * The degree of parallelism denotes how many parallel instances of the user function will be
+ * Gets the parallelism for the operator represented by this optimizer node.
+ * The parallelism denotes how many parallel instances of the operator on will be
* spawned during the execution. If this value is <code>-1</code>, then the system will take
* the default number of parallel instances.
*
- * @return The degree of parallelism.
+ * @return The parallelism of the operator.
*/
public int getDegreeOfParallelism() {
return this.degreeOfParallelism;
}
/**
- * Sets the degree of parallelism for the contract represented by this optimizer node.
- * The degree of parallelism denotes how many parallel instances of the user function will be
+ * Sets the parallelism for this optimizer node.
+ * The parallelism denotes how many parallel instances of the operator will be
* spawned during the execution. If this value is set to <code>-1</code>, then the system will take
* the default number of parallel instances.
*
- * @param degreeOfParallelism
- * The degree of parallelism to set.
- * @throws IllegalArgumentException
- * If the degree of parallelism is smaller than one and not -1.
+ * @param parallelism The parallelism to set.
+ * @throws IllegalArgumentException If the parallelism is smaller than one and not -1.
*/
- public void setDegreeOfParallelism(int degreeOfParallelism) {
- if (degreeOfParallelism < 1) {
- throw new IllegalArgumentException("Degree of parallelism of " + degreeOfParallelism + " is invalid.");
+ public void setDegreeOfParallelism(int parallelism) {
+ if (parallelism < 1 && parallelism != -1) {
+ throw new IllegalArgumentException("Degree of parallelism of " + parallelism + " is invalid.");
}
- this.degreeOfParallelism = degreeOfParallelism;
+ this.degreeOfParallelism = parallelism;
}
/**
@@ -498,6 +496,15 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
return getOutgoingConnections() != null && getOutgoingConnections().size() > 1;
}
+ public void markAllOutgoingConnectionsAsPipelineBreaking() {
+ if (this.outgoingConnections == null) {
+ throw new IllegalStateException("The outgoing connections have not yet been initialized.");
+ }
+ for (PactConnection conn : getOutgoingConnections()) {
+ conn.markBreaksPipeline();
+ }
+ }
+
// ------------------------------------------------------------------------
// Miscellaneous
// ------------------------------------------------------------------------
@@ -971,10 +978,15 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
}
/**
- * The node IDs are assigned in graph-traversal order (pre-order), hence, each list is sorted by ID in ascending order and
- * all consecutive lists start with IDs in ascending order.
+ * The node IDs are assigned in graph-traversal order (pre-order), hence, each list is
+ * sorted by ID in ascending order and all consecutive lists start with IDs in ascending order.
+ *
+ * @param markJoinedBranchesAsPipelineBreaking True, if the
*/
- protected final boolean mergeLists(List<UnclosedBranchDescriptor> child1open, List<UnclosedBranchDescriptor> child2open, List<UnclosedBranchDescriptor> result) {
+ protected final boolean mergeLists(List<UnclosedBranchDescriptor> child1open,
+ List<UnclosedBranchDescriptor> child2open,
+ List<UnclosedBranchDescriptor> result,
+ boolean markJoinedBranchesAsPipelineBreaking) {
//remove branches which have already been closed
removeClosedBranches(child1open);
@@ -1033,7 +1045,15 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
// if it is the same, add it only once, otherwise process the join of the paths
if (vector1 == vector2) {
result.add(child1open.get(index1));
- } else {
+ }
+ else {
+ // we merge (re-join) a branch
+
+ // mark the branch as a point where we break the pipeline
+ if (markJoinedBranchesAsPipelineBreaking) {
+ currBanchingNode.markAllOutgoingConnectionsAsPipelineBreaking();
+ }
+
if (this.hereJoinedBranches == null) {
this.hereJoinedBranches = new ArrayList<OptimizerNode>(2);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
index 41369c9..c215ae0 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/PactConnection.java
@@ -51,6 +51,8 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
private int maxDepth = -1;
+ private boolean breakPipeline; // whether this connection should break the pipeline due to potential deadlocks
+
/**
* Creates a new Connection between two nodes. The shipping strategy is by default <tt>NONE</tt>.
* The temp mode is by default <tt>NONE</tt>.
@@ -74,10 +76,10 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
* @param shipStrategy
* The shipping strategy.
* @param exchangeMode
- * The data exchange mode (pipelined / batch)
+ * The data exchange mode (pipelined / batch / batch only for shuffles / ... )
*/
public PactConnection(OptimizerNode source, OptimizerNode target,
- ShipStrategyType shipStrategy, ExecutionMode exchangeMode)
+ ShipStrategyType shipStrategy, ExecutionMode exchangeMode)
{
if (source == null || target == null) {
throw new NullPointerException("Source and target must not be null.");
@@ -95,14 +97,14 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
* @param source
* The source node.
*/
- public PactConnection(OptimizerNode source) {
+ public PactConnection(OptimizerNode source, ExecutionMode exchangeMode) {
if (source == null) {
throw new NullPointerException("Source and target must not be null.");
}
this.source = source;
this.target = null;
this.shipStrategy = ShipStrategyType.NONE;
- this.dataExchangeMode = null;
+ this.dataExchangeMode = exchangeMode;
}
/**
@@ -149,12 +151,30 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
*/
public ExecutionMode getDataExchangeMode() {
if (dataExchangeMode == null) {
- throw new IllegalStateException("This connection does not have a data exchange");
+ throw new IllegalStateException("This connection does not have the data exchange mode set");
}
return dataExchangeMode;
}
/**
+ * Marks that this connection should do a decoupled data exchange (such as batched)
+ * rather then pipeline data. Connections are marked as pipeline breakers to avoid
+ * deadlock situations.
+ */
+ public void markBreaksPipeline() {
+ this.breakPipeline = true;
+ }
+
+ /**
+ * Checks whether this connection is marked to break the pipeline.
+ *
+ * @return True, if this connection is marked to break the pipeline, false otherwise.
+ */
+ public boolean isBreakingPipeline() {
+ return this.breakPipeline;
+ }
+
+ /**
* Gets the interesting properties object for this pact connection.
* If the interesting properties for this connections have not yet been set,
* this method returns null.
@@ -200,6 +220,8 @@ public class PactConnection implements EstimateProvider, DumpableConnection<Opti
}
// --------------------------------------------------------------------------------------------
+ // Estimates
+ // --------------------------------------------------------------------------------------------
@Override
public long getEstimatedOutputSize() {
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SingleInputNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SingleInputNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SingleInputNode.java
index 70c4291..a23f0e5 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SingleInputNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SingleInputNode.java
@@ -30,6 +30,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.operators.Operator;
import org.apache.flink.api.common.operators.SemanticProperties;
import org.apache.flink.api.common.operators.SingleInputOperator;
@@ -50,17 +51,18 @@ import org.apache.flink.compiler.plan.SingleInputPlanNode;
import org.apache.flink.compiler.plan.PlanNode.SourceAndDamReport;
import org.apache.flink.compiler.util.NoOpUnaryUdfOp;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.util.Visitor;
import com.google.common.collect.Sets;
/**
- * A node in the optimizer's program representation for a PACT with a single input.
+ * A node in the optimizer's program representation for an operation with a single input.
*
- * This class contains all the generic logic for branch handling, interesting properties,
- * and candidate plan enumeration. The subclasses for specific operators simply add logic
- * for cost estimates and specify possible strategies for their realization.
+ * This class contains all the generic logic for handling branching flows, as well as to
+ * enumerate candidate execution plans. The subclasses for specific operators simply add logic
+ * for cost estimates and specify possible strategies for their execution.
*/
public abstract class SingleInputNode extends OptimizerNode {
@@ -73,12 +75,12 @@ public abstract class SingleInputNode extends OptimizerNode {
/**
* Creates a new node with a single input for the optimizer plan.
*
- * @param pactContract The PACT that the node represents.
+ * @param programOperator The PACT that the node represents.
*/
- protected SingleInputNode(SingleInputOperator<?, ?, ?> pactContract) {
- super(pactContract);
+ protected SingleInputNode(SingleInputOperator<?, ?, ?> programOperator) {
+ super(programOperator);
- int[] k = pactContract.getKeyColumns(0);
+ int[] k = programOperator.getKeyColumns(0);
this.keys = k == null || k.length == 0 ? null : new FieldSet(k);
}
@@ -115,7 +117,7 @@ public abstract class SingleInputNode extends OptimizerNode {
}
/**
- * Sets the <tt>PactConnection</tt> through which this node receives its input.
+ * Sets the connection through which this node receives its input.
*
* @param inConn The input connection to set.
*/
@@ -136,7 +138,6 @@ public abstract class SingleInputNode extends OptimizerNode {
}
}
-
@Override
public List<PactConnection> getIncomingConnections() {
return Collections.singletonList(this.inConn);
@@ -145,12 +146,14 @@ public abstract class SingleInputNode extends OptimizerNode {
@Override
public SemanticProperties getSemanticProperties() {
- return ((SingleInputOperator<?,?,?>) getPactContract()).getSemanticProperties();
+ return getPactContract().getSemanticProperties();
}
@Override
- public void setInput(Map<Operator<?>, OptimizerNode> contractToNode) throws CompilerException {
+ public void setInput(Map<Operator<?>, OptimizerNode> contractToNode, ExecutionMode defaultExchangeMode)
+ throws CompilerException
+ {
// see if an internal hint dictates the strategy to use
final Configuration conf = getPactContract().getParameters();
final String shipStrategy = conf.getString(PactCompiler.HINT_SHIP_STRATEGY, null);
@@ -181,7 +184,7 @@ public abstract class SingleInputNode extends OptimizerNode {
throw new CompilerException("Error: Node for '" + getPactContract().getName() + "' has no input.");
} else {
pred = contractToNode.get(children);
- conn = new PactConnection(pred, this);
+ conn = new PactConnection(pred, this, defaultExchangeMode);
if (preSet != null) {
conn.setShipStrategy(preSet);
}
@@ -250,15 +253,19 @@ public abstract class SingleInputNode extends OptimizerNode {
final List<Set<? extends NamedChannel>> broadcastPlanChannels = new ArrayList<Set<? extends NamedChannel>>();
List<PactConnection> broadcastConnections = getBroadcastConnections();
List<String> broadcastConnectionNames = getBroadcastConnectionNames();
+
for (int i = 0; i < broadcastConnections.size(); i++ ) {
PactConnection broadcastConnection = broadcastConnections.get(i);
String broadcastConnectionName = broadcastConnectionNames.get(i);
List<PlanNode> broadcastPlanCandidates = broadcastConnection.getSource().getAlternativePlans(estimator);
- // wrap the plan candidates in named channels
+
+ // wrap the plan candidates in named channels
HashSet<NamedChannel> broadcastChannels = new HashSet<NamedChannel>(broadcastPlanCandidates.size());
for (PlanNode plan: broadcastPlanCandidates) {
- final NamedChannel c = new NamedChannel(broadcastConnectionName, plan);
- c.setShipStrategy(ShipStrategyType.BROADCAST);
+ NamedChannel c = new NamedChannel(broadcastConnectionName, plan);
+ DataExchangeMode exMode = DataExchangeMode.select(broadcastConnection.getDataExchangeMode(),
+ ShipStrategyType.BROADCAST, broadcastConnection.isBreakingPipeline());
+ c.setShipStrategy(ShipStrategyType.BROADCAST, exMode);
broadcastChannels.add(c);
}
broadcastPlanChannels.add(broadcastChannels);
@@ -270,21 +277,24 @@ public abstract class SingleInputNode extends OptimizerNode {
for (OperatorDescriptorSingle ods : getPossibleProperties()) {
pairs.addAll(ods.getPossibleGlobalProperties());
}
- allValidGlobals = (RequestedGlobalProperties[]) pairs.toArray(new RequestedGlobalProperties[pairs.size()]);
+ allValidGlobals = pairs.toArray(new RequestedGlobalProperties[pairs.size()]);
}
final ArrayList<PlanNode> outputPlans = new ArrayList<PlanNode>();
-
+
+ final ExecutionMode executionMode = this.inConn.getDataExchangeMode();
+
final int dop = getDegreeOfParallelism();
final int inDop = getPredecessorNode().getDegreeOfParallelism();
-
final boolean dopChange = inDop != dop;
+ final boolean breaksPipeline = this.inConn.isBreakingPipeline();
+
// create all candidates
for (PlanNode child : subPlans) {
- if(child.getGlobalProperties().isFullyReplicated()) {
+ if (child.getGlobalProperties().isFullyReplicated()) {
// fully replicated input is always locally forwarded if DOP is not changed
- if(dopChange) {
+ if (dopChange) {
// can not continue with this child
childrenSkippedDueToReplicatedInput = true;
continue;
@@ -297,7 +307,7 @@ public abstract class SingleInputNode extends OptimizerNode {
// pick the strategy ourselves
for (RequestedGlobalProperties igps: intGlobal) {
final Channel c = new Channel(child, this.inConn.getMaterializationMode());
- igps.parameterizeChannel(c, dopChange);
+ igps.parameterizeChannel(c, dopChange, executionMode, breaksPipeline);
// if the DOP changed, make sure that we cancel out properties, unless the
// ship strategy preserves/establishes them even under changing DOPs
@@ -320,10 +330,13 @@ public abstract class SingleInputNode extends OptimizerNode {
} else {
// hint fixed the strategy
final Channel c = new Channel(child, this.inConn.getMaterializationMode());
+ final ShipStrategyType shipStrategy = this.inConn.getShipStrategy();
+ final DataExchangeMode exMode = DataExchangeMode.select(executionMode, shipStrategy, breaksPipeline);
+
if (this.keys != null) {
- c.setShipStrategy(this.inConn.getShipStrategy(), this.keys.toFieldList());
+ c.setShipStrategy(shipStrategy, this.keys.toFieldList(), exMode);
} else {
- c.setShipStrategy(this.inConn.getShipStrategy());
+ c.setShipStrategy(shipStrategy, exMode);
}
if (dopChange) {
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SinkJoiner.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SinkJoiner.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SinkJoiner.java
index c153078..088a0f3 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SinkJoiner.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/SinkJoiner.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.typeinfo.NothingTypeInfo;
import org.apache.flink.compiler.DataStatistics;
import org.apache.flink.compiler.operators.OperatorDescriptorDual;
@@ -39,8 +40,8 @@ public class SinkJoiner extends TwoInputNode {
public SinkJoiner(OptimizerNode input1, OptimizerNode input2) {
super(new NoOpBinaryUdfOp<Nothing>(new NothingTypeInfo()));
- PactConnection conn1 = new PactConnection(input1, this);
- PactConnection conn2 = new PactConnection(input2, this);
+ PactConnection conn1 = new PactConnection(input1, this, null, ExecutionMode.PIPELINED);
+ PactConnection conn2 = new PactConnection(input2, this, null, ExecutionMode.PIPELINED);
this.input1 = conn1;
this.input2 = conn2;
@@ -87,7 +88,7 @@ public class SinkJoiner extends TwoInputNode {
List<UnclosedBranchDescriptor> result2 = new ArrayList<UnclosedBranchDescriptor>(pred2branches);
ArrayList<UnclosedBranchDescriptor> result = new ArrayList<UnclosedBranchDescriptor>();
- mergeLists(result1, result2, result);
+ mergeLists(result1, result2, result, false);
this.openBranches = result.isEmpty() ? Collections.<UnclosedBranchDescriptor>emptyList() : result;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/TwoInputNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/TwoInputNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/TwoInputNode.java
index b391890..6faceaa 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/TwoInputNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/TwoInputNode.java
@@ -30,6 +30,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.operators.DualInputOperator;
import org.apache.flink.api.common.operators.Operator;
import org.apache.flink.api.common.operators.SemanticProperties;
@@ -51,6 +52,7 @@ import org.apache.flink.compiler.plan.NamedChannel;
import org.apache.flink.compiler.plan.PlanNode;
import org.apache.flink.compiler.plan.PlanNode.SourceAndDamReport;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
import org.apache.flink.runtime.operators.DamBehavior;
import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
@@ -155,7 +157,7 @@ public abstract class TwoInputNode extends OptimizerNode {
@Override
- public void setInput(Map<Operator<?>, OptimizerNode> contractToNode) {
+ public void setInput(Map<Operator<?>, OptimizerNode> contractToNode, ExecutionMode defaultExecutionMode) {
// see if there is a hint that dictates which shipping strategy to use for BOTH inputs
final Configuration conf = getPactContract().getParameters();
ShipStrategyType preSet1 = null;
@@ -215,7 +217,7 @@ public abstract class TwoInputNode extends OptimizerNode {
}
// get the predecessors
- DualInputOperator<?, ?, ?, ?> contr = (DualInputOperator<?, ?, ?, ?>) getPactContract();
+ DualInputOperator<?, ?, ?, ?> contr = getPactContract();
Operator<?> leftPred = contr.getFirstInput();
Operator<?> rightPred = contr.getSecondInput();
@@ -226,7 +228,7 @@ public abstract class TwoInputNode extends OptimizerNode {
throw new CompilerException("Error: Node for '" + getPactContract().getName() + "' has no input set for first input.");
} else {
pred1 = contractToNode.get(leftPred);
- conn1 = new PactConnection(pred1, this);
+ conn1 = new PactConnection(pred1, this, defaultExecutionMode);
if (preSet1 != null) {
conn1.setShipStrategy(preSet1);
}
@@ -242,7 +244,7 @@ public abstract class TwoInputNode extends OptimizerNode {
throw new CompilerException("Error: Node for '" + getPactContract().getName() + "' has no input set for second input.");
} else {
pred2 = contractToNode.get(rightPred);
- conn2 = new PactConnection(pred2, this);
+ conn2 = new PactConnection(pred2, this, defaultExecutionMode);
if (preSet2 != null) {
conn2.setShipStrategy(preSet2);
}
@@ -314,15 +316,19 @@ public abstract class TwoInputNode extends OptimizerNode {
final List<Set<? extends NamedChannel>> broadcastPlanChannels = new ArrayList<Set<? extends NamedChannel>>();
List<PactConnection> broadcastConnections = getBroadcastConnections();
List<String> broadcastConnectionNames = getBroadcastConnectionNames();
+
for (int i = 0; i < broadcastConnections.size(); i++ ) {
PactConnection broadcastConnection = broadcastConnections.get(i);
String broadcastConnectionName = broadcastConnectionNames.get(i);
List<PlanNode> broadcastPlanCandidates = broadcastConnection.getSource().getAlternativePlans(estimator);
- // wrap the plan candidates in named channels
+
+ // wrap the plan candidates in named channels
HashSet<NamedChannel> broadcastChannels = new HashSet<NamedChannel>(broadcastPlanCandidates.size());
for (PlanNode plan: broadcastPlanCandidates) {
final NamedChannel c = new NamedChannel(broadcastConnectionName, plan);
- c.setShipStrategy(ShipStrategyType.BROADCAST);
+ DataExchangeMode exMode = DataExchangeMode.select(broadcastConnection.getDataExchangeMode(),
+ ShipStrategyType.BROADCAST, broadcastConnection.isBreakingPipeline());
+ c.setShipStrategy(ShipStrategyType.BROADCAST, exMode);
broadcastChannels.add(c);
}
broadcastPlanChannels.add(broadcastChannels);
@@ -337,12 +343,15 @@ public abstract class TwoInputNode extends OptimizerNode {
pairsGlob.addAll(ods.getPossibleGlobalProperties());
pairsLoc.addAll(ods.getPossibleLocalProperties());
}
- allGlobalPairs = (GlobalPropertiesPair[]) pairsGlob.toArray(new GlobalPropertiesPair[pairsGlob.size()]);
- allLocalPairs = (LocalPropertiesPair[]) pairsLoc.toArray(new LocalPropertiesPair[pairsLoc.size()]);
+ allGlobalPairs = pairsGlob.toArray(new GlobalPropertiesPair[pairsGlob.size()]);
+ allLocalPairs = pairsLoc.toArray(new LocalPropertiesPair[pairsLoc.size()]);
}
final ArrayList<PlanNode> outputPlans = new ArrayList<PlanNode>();
-
+
+ final ExecutionMode input1Mode = this.input1.getDataExchangeMode();
+ final ExecutionMode input2Mode = this.input2.getDataExchangeMode();
+
final int dop = getDegreeOfParallelism();
final int inDop1 = getFirstPredecessorNode().getDegreeOfParallelism();
final int inDop2 = getSecondPredecessorNode().getDegreeOfParallelism();
@@ -350,15 +359,18 @@ public abstract class TwoInputNode extends OptimizerNode {
final boolean dopChange1 = dop != inDop1;
final boolean dopChange2 = dop != inDop2;
+ final boolean input1breaksPipeline = this.input1.isBreakingPipeline();
+ final boolean input2breaksPipeline = this.input2.isBreakingPipeline();
+
// enumerate all pairwise combination of the children's plans together with
// all possible operator strategy combination
// create all candidates
for (PlanNode child1 : subPlans1) {
- if(child1.getGlobalProperties().isFullyReplicated()) {
+ if (child1.getGlobalProperties().isFullyReplicated()) {
// fully replicated input is always locally forwarded if DOP is not changed
- if(dopChange1) {
+ if (dopChange1) {
// can not continue with this child
childrenSkippedDueToReplicatedInput = true;
continue;
@@ -369,9 +381,9 @@ public abstract class TwoInputNode extends OptimizerNode {
for (PlanNode child2 : subPlans2) {
- if(child2.getGlobalProperties().isFullyReplicated()) {
+ if (child2.getGlobalProperties().isFullyReplicated()) {
// fully replicated input is always locally forwarded if DOP is not changed
- if(dopChange2) {
+ if (dopChange2) {
// can not continue with this child
childrenSkippedDueToReplicatedInput = true;
continue;
@@ -391,19 +403,23 @@ public abstract class TwoInputNode extends OptimizerNode {
final Channel c1 = new Channel(child1, this.input1.getMaterializationMode());
if (this.input1.getShipStrategy() == null) {
// free to choose the ship strategy
- igps1.parameterizeChannel(c1, dopChange1);
+ 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 (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
c1.getGlobalProperties().reset();
}
- } else {
+ }
+ else {
// ship strategy fixed by compiler hint
+ ShipStrategyType shipType = this.input1.getShipStrategy();
+ DataExchangeMode exMode = DataExchangeMode.select(input1Mode, shipType, input1breaksPipeline);
if (this.keys1 != null) {
- c1.setShipStrategy(this.input1.getShipStrategy(), this.keys1.toFieldList());
- } else {
- c1.setShipStrategy(this.input1.getShipStrategy());
+ c1.setShipStrategy(shipType, this.keys1.toFieldList(), exMode);
+ }
+ else {
+ c1.setShipStrategy(shipType, exMode);
}
if (dopChange1) {
@@ -416,7 +432,7 @@ public abstract class TwoInputNode extends OptimizerNode {
final Channel c2 = new Channel(child2, this.input2.getMaterializationMode());
if (this.input2.getShipStrategy() == null) {
// free to choose the ship strategy
- igps2.parameterizeChannel(c2, dopChange2);
+ 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
@@ -425,10 +441,12 @@ public abstract class TwoInputNode extends OptimizerNode {
}
} else {
// ship strategy fixed by compiler hint
+ ShipStrategyType shipType = this.input2.getShipStrategy();
+ DataExchangeMode exMode = DataExchangeMode.select(input2Mode, shipType, input2breaksPipeline);
if (this.keys2 != null) {
- c2.setShipStrategy(this.input2.getShipStrategy(), this.keys2.toFieldList());
+ c2.setShipStrategy(shipType, this.keys2.toFieldList(), exMode);
} else {
- c2.setShipStrategy(this.input2.getShipStrategy());
+ c2.setShipStrategy(shipType, exMode);
}
if (dopChange2) {
@@ -437,7 +455,7 @@ public abstract class TwoInputNode extends OptimizerNode {
}
/* ********************************************************************
- * NOTE: Depending on how we proceed with different partitionings,
+ * NOTE: Depending on how we proceed with different partitioning,
* we might at some point need a compatibility check between
* the pairs of global properties.
* *******************************************************************/
@@ -457,7 +475,8 @@ public abstract class TwoInputNode extends OptimizerNode {
// we form a valid combination, so create the local candidates
// for this
- addLocalCandidates(c1Clone, c2, broadcastPlanChannels, igps1, igps2, outputPlans, allLocalPairs, estimator);
+ addLocalCandidates(c1Clone, c2, broadcastPlanChannels, igps1, igps2,
+ outputPlans, allLocalPairs, estimator);
break outer;
}
}
@@ -484,9 +503,11 @@ public abstract class TwoInputNode extends OptimizerNode {
if(outputPlans.isEmpty()) {
if(childrenSkippedDueToReplicatedInput) {
- throw new CompilerException("No plan meeting the requirements could be created @ " + this + ". Most likely reason: Invalid use of replicated input.");
+ throw new CompilerException("No plan meeting the requirements could be created @ " + this
+ + ". Most likely reason: Invalid use of replicated input.");
} else {
- throw new CompilerException("No plan meeting the requirements could be created @ " + this + ". Most likely reason: Too restrictive plan hints.");
+ throw new CompilerException("No plan meeting the requirements could be created @ " + this
+ + ". Most likely reason: Too restrictive plan hints.");
}
}
@@ -535,9 +556,8 @@ public abstract class TwoInputNode extends OptimizerNode {
// all right, co compatible
instantiate(dps, in1Copy, in2Copy, broadcastPlanChannels, target, estimator, rgps1, rgps2, ilp1, ilp2);
break;
- } else {
- // cannot use this pair, fall through the loop and try the next one
}
+ // else cannot use this pair, fall through the loop and try the next one
}
}
}
@@ -675,17 +695,6 @@ public abstract class TwoInputNode extends OptimizerNode {
}
}
}
-
- /**
- * Checks if the subPlan has a valid outputSize estimation.
- *
- * @param subPlan The subPlan to check.
- *
- * @return {@code True}, if all values are valid, {@code false} otherwise
- */
- protected boolean haveValidOutputEstimates(OptimizerNode subPlan) {
- return subPlan.getEstimatedOutputSize() != -1;
- }
@Override
public void computeUnclosedBranchStack() {
@@ -701,7 +710,7 @@ public abstract class TwoInputNode extends OptimizerNode {
List<UnclosedBranchDescriptor> result2 = getSecondPredecessorNode().getBranchesForParent(getSecondIncomingConnection());
ArrayList<UnclosedBranchDescriptor> inputsMerged = new ArrayList<UnclosedBranchDescriptor>();
- mergeLists(result1, result2, inputsMerged);
+ mergeLists(result1, result2, inputsMerged, true);
// handle the data flow branching for the broadcast inputs
List<UnclosedBranchDescriptor> result = computeUnclosedBranchStackForBroadcastInputs(inputsMerged);
@@ -709,23 +718,9 @@ public abstract class TwoInputNode extends OptimizerNode {
this.openBranches = (result == null || result.isEmpty()) ? Collections.<UnclosedBranchDescriptor>emptyList() : result;
}
- /**
- * Returns the key fields of the given input.
- *
- * @param input The input for which key fields must be returned.
- * @return the key fields of the given input.
- */
- public FieldList getInputKeySet(int input) {
- switch(input) {
- case 0: return keys1;
- case 1: return keys2;
- default: throw new IndexOutOfBoundsException();
- }
- }
-
@Override
public SemanticProperties getSemanticProperties() {
- return ((DualInputOperator<?, ?, ?, ?>) getPactContract()).getSemanticProperties();
+ return getPactContract().getSemanticProperties();
}
// --------------------------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/main/java/org/apache/flink/compiler/dag/WorksetIterationNode.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/WorksetIterationNode.java b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/WorksetIterationNode.java
index 0557633..3ae4976 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/dag/WorksetIterationNode.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/dag/WorksetIterationNode.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Set;
+import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.operators.SemanticProperties;
import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties;
import org.apache.flink.api.common.operators.base.DeltaIterationBase;
@@ -148,7 +149,9 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
this.worksetNode = worksetNode;
}
- public void setNextPartialSolution(OptimizerNode solutionSetDelta, OptimizerNode nextWorkset) {
+ public void setNextPartialSolution(OptimizerNode solutionSetDelta, OptimizerNode nextWorkset,
+ ExecutionMode executionMode) {
+
// check whether the next partial solution is itself the join with
// the partial solution (so we can potentially do direct updates)
if (solutionSetDelta instanceof TwoInputNode) {
@@ -166,7 +169,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
NoOpNode noop = new NoOpNode();
noop.setDegreeOfParallelism(getDegreeOfParallelism());
- PactConnection noOpConn = new PactConnection(nextWorkset, noop);
+ PactConnection noOpConn = new PactConnection(nextWorkset, noop, executionMode);
noop.setIncomingConnection(noOpConn);
nextWorkset.addOutgoingConnection(noOpConn);
@@ -178,7 +181,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
new SolutionSetDeltaOperator(getSolutionSetKeyFields()));
solutionSetDeltaUpdateAux.setDegreeOfParallelism(getDegreeOfParallelism());
- PactConnection conn = new PactConnection(solutionSetDelta, solutionSetDeltaUpdateAux);
+ PactConnection conn = new PactConnection(solutionSetDelta, solutionSetDeltaUpdateAux, executionMode);
solutionSetDeltaUpdateAux.setIncomingConnection(conn);
solutionSetDelta.addOutgoingConnection(conn);
@@ -186,8 +189,10 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
this.nextWorkset = nextWorkset;
this.singleRoot = new SingleRootJoiner();
- this.solutionSetDeltaRootConnection = new PactConnection(solutionSetDeltaUpdateAux, this.singleRoot);
- this.nextWorksetRootConnection = new PactConnection(nextWorkset, this.singleRoot);
+ this.solutionSetDeltaRootConnection = new PactConnection(solutionSetDeltaUpdateAux,
+ this.singleRoot, executionMode);
+
+ this.nextWorksetRootConnection = new PactConnection(nextWorkset, this.singleRoot, executionMode);
this.singleRoot.setInputs(this.solutionSetDeltaRootConnection, this.nextWorksetRootConnection);
solutionSetDeltaUpdateAux.addOutgoingConnection(this.solutionSetDeltaRootConnection);
@@ -350,30 +355,37 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
GlobalProperties atEndGlobal = candidate.getGlobalProperties();
LocalProperties atEndLocal = candidate.getLocalProperties();
- FeedbackPropertiesMeetRequirementsReport report = candidate.checkPartialSolutionPropertiesMet(wspn, atEndGlobal, atEndLocal);
+ FeedbackPropertiesMeetRequirementsReport report = candidate.checkPartialSolutionPropertiesMet(wspn,
+ atEndGlobal, atEndLocal);
+
if (report == FeedbackPropertiesMeetRequirementsReport.NO_PARTIAL_SOLUTION) {
; // depends only through broadcast variable on the workset solution
}
else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) {
// attach a no-op node through which we create the properties of the original input
Channel toNoOp = new Channel(candidate);
- globPropsReqWorkset.parameterizeChannel(toNoOp, false);
+ globPropsReqWorkset.parameterizeChannel(toNoOp, false,
+ nextWorksetRootConnection.getDataExchangeMode(), false);
locPropsReqWorkset.parameterizeChannel(toNoOp);
- UnaryOperatorNode rebuildWorksetPropertiesNode = new UnaryOperatorNode("Rebuild Workset Properties", FieldList.EMPTY_LIST);
+ UnaryOperatorNode rebuildWorksetPropertiesNode = new UnaryOperatorNode("Rebuild Workset Properties",
+ FieldList.EMPTY_LIST);
rebuildWorksetPropertiesNode.setDegreeOfParallelism(candidate.getDegreeOfParallelism());
- SingleInputPlanNode rebuildWorksetPropertiesPlanNode = new SingleInputPlanNode(rebuildWorksetPropertiesNode, "Rebuild Workset Properties", toNoOp, DriverStrategy.UNARY_NO_OP);
- rebuildWorksetPropertiesPlanNode.initProperties(toNoOp.getGlobalProperties(), toNoOp.getLocalProperties());
+ SingleInputPlanNode rebuildWorksetPropertiesPlanNode = new SingleInputPlanNode(
+ rebuildWorksetPropertiesNode, "Rebuild Workset Properties",
+ toNoOp, DriverStrategy.UNARY_NO_OP);
+ rebuildWorksetPropertiesPlanNode.initProperties(toNoOp.getGlobalProperties(),
+ toNoOp.getLocalProperties());
estimator.costOperator(rebuildWorksetPropertiesPlanNode);
GlobalProperties atEndGlobalModified = rebuildWorksetPropertiesPlanNode.getGlobalProperties();
LocalProperties atEndLocalModified = rebuildWorksetPropertiesPlanNode.getLocalProperties();
if (!(atEndGlobalModified.equals(atEndGlobal) && atEndLocalModified.equals(atEndLocal))) {
- FeedbackPropertiesMeetRequirementsReport report2 = candidate.checkPartialSolutionPropertiesMet(wspn, atEndGlobalModified, atEndLocalModified);
-
+ FeedbackPropertiesMeetRequirementsReport report2 = candidate.checkPartialSolutionPropertiesMet(
+ wspn, atEndGlobalModified, atEndLocalModified);
if (report2 != FeedbackPropertiesMeetRequirementsReport.NOT_MET) {
newCandidates.add(rebuildWorksetPropertiesPlanNode);
}
@@ -393,13 +405,12 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
}
// sanity check the solution set delta
- for (Iterator<PlanNode> deltaPlans = solutionSetDeltaCandidates.iterator(); deltaPlans.hasNext(); ) {
- SingleInputPlanNode candidate = (SingleInputPlanNode) deltaPlans.next();
+ for (PlanNode solutionSetDeltaCandidate : solutionSetDeltaCandidates) {
+ SingleInputPlanNode candidate = (SingleInputPlanNode) solutionSetDeltaCandidate;
GlobalProperties gp = candidate.getGlobalProperties();
-
+
if (gp.getPartitioning() != PartitioningProperty.HASH_PARTITIONED || gp.getPartitioningFields() == null ||
- !gp.getPartitioningFields().equals(this.solutionSetKeyFields))
- {
+ !gp.getPartitioningFields().equals(this.solutionSetKeyFields)) {
throw new CompilerException("Bug: The solution set delta is not partitioned.");
}
}
@@ -422,10 +433,14 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
boolean immediateDeltaUpdate;
// check whether we need a dedicated solution set delta operator, or whether we can update on the fly
- if (siSolutionDeltaCandidate.getInput().getShipStrategy() == ShipStrategyType.FORWARD && this.solutionDeltaImmediatelyAfterSolutionJoin) {
+ if (siSolutionDeltaCandidate.getInput().getShipStrategy() == ShipStrategyType.FORWARD &&
+ this.solutionDeltaImmediatelyAfterSolutionJoin)
+ {
// we do not need this extra node. we can make the predecessor the delta
// sanity check the node and connection
- if (siSolutionDeltaCandidate.getDriverStrategy() != DriverStrategy.UNARY_NO_OP || siSolutionDeltaCandidate.getInput().getLocalStrategy() != LocalStrategy.NONE) {
+ if (siSolutionDeltaCandidate.getDriverStrategy() != DriverStrategy.UNARY_NO_OP ||
+ siSolutionDeltaCandidate.getInput().getLocalStrategy() != LocalStrategy.NONE)
+ {
throw new CompilerException("Invalid Solution set delta node.");
}
@@ -438,8 +453,9 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
immediateDeltaUpdate = false;
}
- WorksetIterationPlanNode wsNode = new WorksetIterationPlanNode(
- this, "WorksetIteration ("+this.getPactContract().getName()+")", solutionSetIn, worksetIn, sspn, wspn, worksetCandidate, solutionSetCandidate);
+ WorksetIterationPlanNode wsNode = new WorksetIterationPlanNode(this,
+ "WorksetIteration ("+this.getPactContract().getName()+")", solutionSetIn,
+ worksetIn, sspn, wspn, worksetCandidate, solutionSetCandidate);
wsNode.setImmediateSolutionSetUpdate(immediateDeltaUpdate);
wsNode.initProperties(gp, lp);
target.add(wsNode);
@@ -453,7 +469,6 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
if (this.openBranches != null) {
return;
}
-
// IMPORTANT: First compute closed branches from the two inputs
// we need to do this because the runtime iteration head effectively joins
@@ -464,13 +479,13 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
List<UnclosedBranchDescriptor> result2 = getSecondPredecessorNode().getBranchesForParent(getSecondIncomingConnection());
ArrayList<UnclosedBranchDescriptor> inputsMerged1 = new ArrayList<UnclosedBranchDescriptor>();
- mergeLists(result1, result2, inputsMerged1); // this method also sets which branches are joined here (in the head)
+ mergeLists(result1, result2, inputsMerged1, true); // this method also sets which branches are joined here (in the head)
addClosedBranches(getSingleRootOfStepFunction().closedBranchingNodes);
ArrayList<UnclosedBranchDescriptor> inputsMerged2 = new ArrayList<UnclosedBranchDescriptor>();
List<UnclosedBranchDescriptor> result3 = getSingleRootOfStepFunction().openBranches;
- mergeLists(inputsMerged1, result3, inputsMerged2);
+ mergeLists(inputsMerged1, result3, inputsMerged2, true);
// handle the data flow branching for the broadcast inputs
List<UnclosedBranchDescriptor> result = computeUnclosedBranchStackForBroadcastInputs(inputsMerged2);
[2/8] flink git commit: [FLINK-1671] [tests] Rename "ExecutionMode"
in MultiProgramTestBase to "TestExecutionMode"
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala
index c2cd20a..9e147b8 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3
import org.apache.flink.configuration.Configuration
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -34,7 +34,7 @@ import scala.collection.JavaConverters._
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class ReduceITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class ReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SumMinMaxITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SumMinMaxITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SumMinMaxITCase.scala
index 6071569..d57c6e2d7f 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SumMinMaxITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/SumMinMaxITCase.scala
@@ -21,7 +21,7 @@ package org.apache.flink.api.scala.operators
import org.apache.flink.api.scala.ExecutionEnvironment
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit.{Test, After, Before, Rule}
import org.junit.rules.TemporaryFolder
@@ -32,7 +32,7 @@ import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class SumMinMaxITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class SumMinMaxITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
index 255cce9..2cf3ab3 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
@@ -19,7 +19,7 @@ package org.apache.flink.api.scala.operators
import org.apache.flink.api.scala.util.CollectionDataSets
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit._
import org.junit.rules.TemporaryFolder
@@ -29,7 +29,7 @@ import org.junit.runners.Parameterized
import org.apache.flink.api.scala._
@RunWith(classOf[Parameterized])
-class UnionITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class UnionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
private var resultPath: String = null
private var expected: String = null
private val _tempFolder = new TemporaryFolder()
http://git-wip-us.apache.org/repos/asf/flink/blob/47827702/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesITCase.scala
index 57ef08e..63ec2a4 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesITCase.scala
@@ -19,7 +19,7 @@ package org.apache.flink.api.scala.runtime
import org.apache.flink.api.scala.ExecutionEnvironment
import org.apache.flink.core.fs.FileSystem.WriteMode
-import org.apache.flink.test.util.MultipleProgramsTestBase.ExecutionMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{MultipleProgramsTestBase}
import org.junit._
import org.junit.rules.TemporaryFolder
@@ -31,7 +31,7 @@ import org.junit.runners.Parameterized
import scala.util.{Failure, Success}
@RunWith(classOf[Parameterized])
-class ScalaSpecialTypesITCase(mode: ExecutionMode) extends MultipleProgramsTestBase(mode) {
+class ScalaSpecialTypesITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
val _tempFolder = new TemporaryFolder()
[5/8] flink git commit: [FLINK-1671] [optimizer] Add data exchange
mode to optimizer classes
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeForwardTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeForwardTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeForwardTest.java
new file mode 100644
index 0000000..e550749
--- /dev/null
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeForwardTest.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.compiler.dataexchange;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.compiler.CompilerTestBase;
+import org.apache.flink.compiler.plan.OptimizedPlan;
+import org.apache.flink.compiler.plan.SingleInputPlanNode;
+import org.apache.flink.compiler.plan.SinkPlanNode;
+import org.apache.flink.compiler.testfunctions.IdentityKeyExtractor;
+import org.apache.flink.compiler.testfunctions.Top1GroupReducer;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * This test verifies that the optimizer assigns the correct
+ * data exchange mode to a simple forward / shuffle plan.
+ *
+ * <pre>
+ * (source) -> (map) -> (filter) -> (groupBy / reduce)
+ * </pre>
+ */
+@SuppressWarnings("serial")
+public class DataExchangeModeForwardTest extends CompilerTestBase {
+
+
+ @Test
+ public void testPipelinedForced() {
+ // PIPELINED_FORCED should result in pipelining all the way
+ verifySimpleForwardPlan(ExecutionMode.PIPELINED_FORCED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED);
+ }
+
+ @Test
+ public void testPipelined() {
+ // PIPELINED should result in pipelining all the way
+ verifySimpleForwardPlan(ExecutionMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED);
+ }
+
+ @Test
+ public void testBatch() {
+ // BATCH should result in batching the shuffle all the way
+ verifySimpleForwardPlan(ExecutionMode.BATCH,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.BATCH, DataExchangeMode.PIPELINED);
+ }
+
+ @Test
+ public void testBatchForced() {
+ // BATCH_FORCED should result in batching all the way
+ verifySimpleForwardPlan(ExecutionMode.BATCH_FORCED,
+ DataExchangeMode.BATCH, DataExchangeMode.BATCH,
+ DataExchangeMode.BATCH, DataExchangeMode.PIPELINED,
+ DataExchangeMode.BATCH, DataExchangeMode.BATCH);
+ }
+
+ private void verifySimpleForwardPlan(ExecutionMode execMode,
+ DataExchangeMode toMap,
+ DataExchangeMode toFilter,
+ DataExchangeMode toKeyExtractor,
+ DataExchangeMode toCombiner,
+ DataExchangeMode toReduce,
+ DataExchangeMode toSink)
+ {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ env.getConfig().setExecutionMode(execMode);
+
+ DataSet<String> dataSet = env.readTextFile("/never/accessed");
+ dataSet
+ .map(new MapFunction<String, Integer>() {
+ @Override
+ public Integer map(String value) {
+ return 0;
+ }
+ })
+ .filter(new FilterFunction<Integer>() {
+ @Override
+ public boolean filter(Integer value) {
+ return false;
+ }
+ })
+ .groupBy(new IdentityKeyExtractor<Integer>())
+ .reduceGroup(new Top1GroupReducer<Integer>())
+ .output(new DiscardingOutputFormat<Integer>());
+
+ OptimizedPlan optPlan = compileNoStats(env.createProgramPlan());
+ SinkPlanNode sinkNode = optPlan.getDataSinks().iterator().next();
+
+ SingleInputPlanNode reduceNode = (SingleInputPlanNode) sinkNode.getPredecessor();
+ SingleInputPlanNode combineNode = (SingleInputPlanNode) reduceNode.getPredecessor();
+ SingleInputPlanNode keyExtractorNode = (SingleInputPlanNode) combineNode.getPredecessor();
+
+ SingleInputPlanNode filterNode = (SingleInputPlanNode) keyExtractorNode.getPredecessor();
+ SingleInputPlanNode mapNode = (SingleInputPlanNode) filterNode.getPredecessor();
+
+ assertEquals(toMap, mapNode.getInput().getDataExchangeMode());
+ assertEquals(toFilter, filterNode.getInput().getDataExchangeMode());
+ assertEquals(toKeyExtractor, keyExtractorNode.getInput().getDataExchangeMode());
+ assertEquals(toCombiner, combineNode.getInput().getDataExchangeMode());
+ assertEquals(toReduce, reduceNode.getInput().getDataExchangeMode());
+ assertEquals(toSink, sinkNode.getInput().getDataExchangeMode());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeOpenBranchingTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeOpenBranchingTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeOpenBranchingTest.java
new file mode 100644
index 0000000..e4e0cba
--- /dev/null
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/DataExchangeModeOpenBranchingTest.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.compiler.dataexchange;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.compiler.CompilerTestBase;
+import org.apache.flink.compiler.plan.DualInputPlanNode;
+import org.apache.flink.compiler.plan.OptimizedPlan;
+import org.apache.flink.compiler.plan.SingleInputPlanNode;
+import org.apache.flink.compiler.plan.SinkPlanNode;
+import org.apache.flink.runtime.io.network.DataExchangeMode;
+import org.junit.Test;
+
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * This test checks the correct assignment of the DataExchangeMode to
+ * connections for programs that branch, but do not re-join the branches.
+ *
+ * <pre>
+ * /---> (filter) -> (sink)
+ * /
+ * /
+ * (source) -> (map) -----------------\
+ * \ (join) -> (sink)
+ * \ (source) --/
+ * \
+ * \
+ * \-> (sink)
+ * </pre>
+ */
+@SuppressWarnings({"serial", "unchecked"})
+public class DataExchangeModeOpenBranchingTest extends CompilerTestBase {
+
+ @Test
+ public void testPipelinedForced() {
+ // PIPELINED_FORCED should result in pipelining all the way
+ verifyBranchigPlan(ExecutionMode.PIPELINED_FORCED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED);
+ }
+
+ @Test
+ public void testPipelined() {
+ // PIPELINED should result in pipelining all the way
+ verifyBranchigPlan(ExecutionMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED);
+ }
+
+ @Test
+ public void testBatch() {
+ // BATCH should result in batching the shuffle all the way
+ verifyBranchigPlan(ExecutionMode.BATCH,
+ DataExchangeMode.PIPELINED, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED, DataExchangeMode.BATCH,
+ DataExchangeMode.BATCH, DataExchangeMode.PIPELINED,
+ DataExchangeMode.PIPELINED);
+ }
+
+ @Test
+ public void testBatchForced() {
+ // BATCH_FORCED should result in batching all the way
+ verifyBranchigPlan(ExecutionMode.BATCH_FORCED,
+ DataExchangeMode.BATCH, DataExchangeMode.BATCH,
+ DataExchangeMode.BATCH, DataExchangeMode.BATCH,
+ DataExchangeMode.BATCH, DataExchangeMode.BATCH,
+ DataExchangeMode.BATCH);
+ }
+
+ private void verifyBranchigPlan(ExecutionMode execMode,
+ DataExchangeMode toMap,
+ DataExchangeMode toFilter,
+ DataExchangeMode toFilterSink,
+ DataExchangeMode toJoin1,
+ DataExchangeMode toJoin2,
+ DataExchangeMode toJoinSink,
+ DataExchangeMode toDirectSink)
+ {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ env.getConfig().setExecutionMode(execMode);
+
+ DataSet<Tuple2<Long, Long>> data = env.generateSequence(1, 100000)
+ .map(new MapFunction<Long, Tuple2<Long, Long>>() {
+ @Override
+ public Tuple2<Long, Long> map(Long value) {
+ return new Tuple2<Long, Long>(value, value);
+ }
+ });
+
+ // output 1
+ data
+ .filter(new FilterFunction<Tuple2<Long, Long>>() {
+ @Override
+ public boolean filter(Tuple2<Long, Long> value) {
+ return false;
+ }
+ })
+ .output(new DiscardingOutputFormat<Tuple2<Long, Long>>()).name("sink1");
+
+ // output 2 does a join before a join
+ data
+ .join(env.fromElements(new Tuple2<Long, Long>(1L, 2L)))
+ .where(1)
+ .equalTo(0)
+ .output(new DiscardingOutputFormat<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>>()).name("sink2");
+
+ // output 3 is direct
+ data
+ .output(new DiscardingOutputFormat<Tuple2<Long, Long>>()).name("sink3");
+
+ OptimizedPlan optPlan = compileNoStats(env.createProgramPlan());
+
+ SinkPlanNode filterSink = findSink(optPlan.getDataSinks(), "sink1");
+ SinkPlanNode joinSink = findSink(optPlan.getDataSinks(), "sink2");
+ SinkPlanNode directSink = findSink(optPlan.getDataSinks(), "sink3");
+
+ SingleInputPlanNode filterNode = (SingleInputPlanNode) filterSink.getPredecessor();
+ SingleInputPlanNode mapNode = (SingleInputPlanNode) filterNode.getPredecessor();
+
+ DualInputPlanNode joinNode = (DualInputPlanNode) joinSink.getPredecessor();
+ assertEquals(mapNode, joinNode.getInput1().getSource());
+
+ assertEquals(mapNode, directSink.getPredecessor());
+
+ assertEquals(toFilterSink, filterSink.getInput().getDataExchangeMode());
+ assertEquals(toJoinSink, joinSink.getInput().getDataExchangeMode());
+ assertEquals(toDirectSink, directSink.getInput().getDataExchangeMode());
+
+ assertEquals(toMap, mapNode.getInput().getDataExchangeMode());
+ assertEquals(toFilter, filterNode.getInput().getDataExchangeMode());
+
+ assertEquals(toJoin1, joinNode.getInput1().getDataExchangeMode());
+ assertEquals(toJoin2, joinNode.getInput2().getDataExchangeMode());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private SinkPlanNode findSink(Collection<SinkPlanNode> collection, String name) {
+ for (SinkPlanNode node : collection) {
+ String nodeName = node.getOptimizerNode().getPactContract().getName();
+ if (nodeName != null && nodeName.equals(name)) {
+ return node;
+ }
+ }
+
+ throw new IllegalArgumentException("No node with that name was found.");
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/PipelineBreakingTest.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/PipelineBreakingTest.java b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/PipelineBreakingTest.java
new file mode 100644
index 0000000..f6a77c6
--- /dev/null
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/dataexchange/PipelineBreakingTest.java
@@ -0,0 +1,320 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.compiler.dataexchange;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.compiler.PactCompiler;
+import org.apache.flink.compiler.dag.DataSinkNode;
+import org.apache.flink.compiler.dag.OptimizerNode;
+import org.apache.flink.compiler.dag.SingleInputNode;
+import org.apache.flink.compiler.dag.SinkJoiner;
+import org.apache.flink.compiler.dag.TwoInputNode;
+import org.apache.flink.compiler.testfunctions.DummyCoGroupFunction;
+import org.apache.flink.compiler.testfunctions.DummyFlatJoinFunction;
+import org.apache.flink.compiler.testfunctions.IdentityFlatMapper;
+import org.apache.flink.compiler.testfunctions.IdentityKeyExtractor;
+import org.apache.flink.compiler.testfunctions.SelectOneReducer;
+import org.apache.flink.compiler.testfunctions.Top1GroupReducer;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ * This test checks whether connections are correctly marked as pipelined breaking.
+ */
+@SuppressWarnings("serial")
+public class PipelineBreakingTest {
+
+ /**
+ * Tests that no pipeline breakers are inserted into a simple forward
+ * pipeline.
+ *
+ * <pre>
+ * (source) -> (map) -> (filter) -> (groupBy / reduce)
+ * </pre>
+ */
+ @Test
+ public void testSimpleForwardPlan() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<String> dataSet = env.readTextFile("/never/accessed");
+ dataSet
+ .map(new MapFunction<String, Integer>() {
+ @Override
+ public Integer map(String value) {
+ return 0;
+ }
+ })
+ .filter(new FilterFunction<Integer>() {
+ @Override
+ public boolean filter(Integer value) {
+ return false;
+ }
+ })
+ .groupBy(new IdentityKeyExtractor<Integer>())
+ .reduceGroup(new Top1GroupReducer<Integer>())
+ .output(new DiscardingOutputFormat<Integer>());
+
+ DataSinkNode sinkNode = convertPlan(env.createProgramPlan()).get(0);
+
+ SingleInputNode reduceNode = (SingleInputNode) sinkNode.getPredecessorNode();
+ SingleInputNode keyExtractorNode = (SingleInputNode) reduceNode.getPredecessorNode();
+
+ SingleInputNode filterNode = (SingleInputNode) keyExtractorNode.getPredecessorNode();
+ SingleInputNode mapNode = (SingleInputNode) filterNode.getPredecessorNode();
+
+ assertFalse(sinkNode.getInputConnection().isBreakingPipeline());
+ assertFalse(reduceNode.getIncomingConnection().isBreakingPipeline());
+ assertFalse(keyExtractorNode.getIncomingConnection().isBreakingPipeline());
+ assertFalse(filterNode.getIncomingConnection().isBreakingPipeline());
+ assertFalse(mapNode.getIncomingConnection().isBreakingPipeline());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Tests that branching plans, where the branches are not re-joined,
+ * do not place pipeline breakers.
+ *
+ * <pre>
+ * /---> (filter) -> (sink)
+ * /
+ * /
+ * (source) -> (map) -----------------\
+ * \ (join) -> (sink)
+ * \ (source) --/
+ * \
+ * \
+ * \-> (sink)
+ * </pre>
+ */
+ @Test
+ public void testBranchingPlanNotReJoined() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<Integer> data = env.readTextFile("/never/accessed")
+ .map(new MapFunction<String, Integer>() {
+ @Override
+ public Integer map(String value) {
+ return 0;
+ }
+ });
+
+ // output 1
+ data
+ .filter(new FilterFunction<Integer>() {
+ @Override
+ public boolean filter(Integer value) {
+ return false;
+ }
+ })
+ .output(new DiscardingOutputFormat<Integer>());
+
+ // output 2 does a join before a join
+ data
+ .join(env.fromElements(1, 2, 3, 4))
+ .where(new IdentityKeyExtractor<Integer>())
+ .equalTo(new IdentityKeyExtractor<Integer>())
+ .output(new DiscardingOutputFormat<Tuple2<Integer, Integer>>());
+
+ // output 3 is direct
+ data
+ .output(new DiscardingOutputFormat<Integer>());
+
+ List<DataSinkNode> sinks = convertPlan(env.createProgramPlan());
+
+ // gather the optimizer DAG nodes
+
+ DataSinkNode sinkAfterFilter = sinks.get(0);
+ DataSinkNode sinkAfterJoin = sinks.get(1);
+ DataSinkNode sinkDirect = sinks.get(2);
+
+ SingleInputNode filterNode = (SingleInputNode) sinkAfterFilter.getPredecessorNode();
+ SingleInputNode mapNode = (SingleInputNode) filterNode.getPredecessorNode();
+
+ TwoInputNode joinNode = (TwoInputNode) sinkAfterJoin.getPredecessorNode();
+ SingleInputNode joinInput = (SingleInputNode) joinNode.getSecondPredecessorNode();
+
+ // verify the non-pipeline breaking status
+
+ assertFalse(sinkAfterFilter.getInputConnection().isBreakingPipeline());
+ assertFalse(sinkAfterJoin.getInputConnection().isBreakingPipeline());
+ assertFalse(sinkDirect.getInputConnection().isBreakingPipeline());
+
+ assertFalse(filterNode.getIncomingConnection().isBreakingPipeline());
+ assertFalse(mapNode.getIncomingConnection().isBreakingPipeline());
+
+ assertFalse(joinNode.getFirstIncomingConnection().isBreakingPipeline());
+ assertFalse(joinNode.getSecondIncomingConnection().isBreakingPipeline());
+ assertFalse(joinInput.getIncomingConnection().isBreakingPipeline());
+
+ // some other sanity checks on the plan construction (cannot hurt)
+
+ assertEquals(mapNode, ((SingleInputNode) joinNode.getFirstPredecessorNode()).getPredecessorNode());
+ assertEquals(mapNode, sinkDirect.getPredecessorNode());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Tests that branches that are re-joined have place pipeline breakers.
+ *
+ * <pre>
+ * /-> (sink)
+ * /
+ * /-> (reduce) -+ /-> (flatmap) -> (sink)
+ * / \ /
+ * (source) -> (map) - (join) -+-----\
+ * \ / \
+ * \-> (filter) -+ \
+ * \ (co group) -> (sink)
+ * \ /
+ * \-> (reduce) - /
+ * </pre>
+ */
+ @Test
+ public void testReJoinedBranches() {
+ try {
+ // build a test program
+
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<Tuple2<Long, Long>> data = env.fromElements(33L, 44L)
+ .map(new MapFunction<Long, Tuple2<Long, Long>>() {
+ @Override
+ public Tuple2<Long, Long> map(Long value) {
+ return new Tuple2<Long, Long>(value, value);
+ }
+ });
+
+ DataSet<Tuple2<Long, Long>> reduced = data.groupBy(0).reduce(new SelectOneReducer<Tuple2<Long, Long>>());
+ reduced.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+
+ DataSet<Tuple2<Long, Long>> filtered = data.filter(new FilterFunction<Tuple2<Long, Long>>() {
+ @Override
+ public boolean filter(Tuple2<Long, Long> value) throws Exception {
+ return false;
+ }
+ });
+
+ DataSet<Tuple2<Long, Long>> joined = reduced.join(filtered)
+ .where(1).equalTo(1)
+ .with(new DummyFlatJoinFunction<Tuple2<Long, Long>>());
+
+ joined.flatMap(new IdentityFlatMapper<Tuple2<Long, Long>>())
+ .output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+
+ joined.coGroup(filtered.groupBy(1).reduceGroup(new Top1GroupReducer<Tuple2<Long, Long>>()))
+ .where(0).equalTo(0)
+ .with(new DummyCoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>())
+ .output(new DiscardingOutputFormat<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>>());
+
+ List<DataSinkNode> sinks = convertPlan(env.createProgramPlan());
+
+ // gather the optimizer DAG nodes
+
+ DataSinkNode sinkAfterReduce = sinks.get(0);
+ DataSinkNode sinkAfterFlatMap = sinks.get(1);
+ DataSinkNode sinkAfterCoGroup = sinks.get(2);
+
+ SingleInputNode reduceNode = (SingleInputNode) sinkAfterReduce.getPredecessorNode();
+ SingleInputNode mapNode = (SingleInputNode) reduceNode.getPredecessorNode();
+
+ SingleInputNode flatMapNode = (SingleInputNode) sinkAfterFlatMap.getPredecessorNode();
+ TwoInputNode joinNode = (TwoInputNode) flatMapNode.getPredecessorNode();
+ SingleInputNode filterNode = (SingleInputNode) joinNode.getSecondPredecessorNode();
+
+ TwoInputNode coGroupNode = (TwoInputNode) sinkAfterCoGroup.getPredecessorNode();
+ SingleInputNode otherReduceNode = (SingleInputNode) coGroupNode.getSecondPredecessorNode();
+
+ // test sanity checks (that we constructed the DAG correctly)
+
+ assertEquals(reduceNode, joinNode.getFirstPredecessorNode());
+ assertEquals(mapNode, filterNode.getPredecessorNode());
+ assertEquals(joinNode, coGroupNode.getFirstPredecessorNode());
+ assertEquals(filterNode, otherReduceNode.getPredecessorNode());
+
+ // verify the pipeline breaking status
+
+ assertFalse(sinkAfterReduce.getInputConnection().isBreakingPipeline());
+ assertFalse(sinkAfterFlatMap.getInputConnection().isBreakingPipeline());
+ assertFalse(sinkAfterCoGroup.getInputConnection().isBreakingPipeline());
+
+ assertFalse(mapNode.getIncomingConnection().isBreakingPipeline());
+ assertFalse(flatMapNode.getIncomingConnection().isBreakingPipeline());
+ assertFalse(joinNode.getFirstIncomingConnection().isBreakingPipeline());
+ assertFalse(coGroupNode.getFirstIncomingConnection().isBreakingPipeline());
+ assertFalse(coGroupNode.getSecondIncomingConnection().isBreakingPipeline());
+
+ // these should be pipeline breakers
+ assertTrue(reduceNode.getIncomingConnection().isBreakingPipeline());
+ assertTrue(filterNode.getIncomingConnection().isBreakingPipeline());
+ assertTrue(otherReduceNode.getIncomingConnection().isBreakingPipeline());
+ assertTrue(joinNode.getSecondIncomingConnection().isBreakingPipeline());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private static List<DataSinkNode> convertPlan(Plan p) {
+ PactCompiler.GraphCreatingVisitor dagCreator =
+ new PactCompiler.GraphCreatingVisitor(17, p.getExecutionConfig().getExecutionMode());
+
+ // create the DAG
+ p.accept(dagCreator);
+ List<DataSinkNode> sinks = dagCreator.getSinks();
+
+ // build a single root and run the branch tracking logic
+ OptimizerNode rootNode;
+ if (sinks.size() == 1) {
+ rootNode = sinks.get(0);
+ }
+ else {
+ Iterator<DataSinkNode> iter = sinks.iterator();
+ rootNode = iter.next();
+
+ while (iter.hasNext()) {
+ rootNode = new SinkJoiner(rootNode, iter.next());
+ }
+ }
+ rootNode.accept(new PactCompiler.IdAndEstimatesVisitor(null));
+ rootNode.accept(new PactCompiler.BranchesVisitor());
+
+ return sinks;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityFlatMapper.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityFlatMapper.java b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityFlatMapper.java
new file mode 100644
index 0000000..5fe32b4
--- /dev/null
+++ b/flink-compiler/src/test/java/org/apache/flink/compiler/testfunctions/IdentityFlatMapper.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.compiler.testfunctions;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+
+public class IdentityFlatMapper<T> implements FlatMapFunction<T, T> {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void flatMap(T value, Collector<T> out) {}
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
index e99cac7..f75b797 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.api.common.operators;
import java.util.ArrayList;
@@ -26,9 +25,10 @@ import org.apache.flink.api.common.operators.util.FieldSet;
import org.apache.flink.types.Key;
/**
- *
+ * This class represents an ordering on a set of fields. It specifies the fields and order direction
+ * (ascending, descending).
*/
-public class Ordering {
+public class Ordering implements Cloneable {
protected FieldList indexes = new FieldList();
@@ -212,9 +212,6 @@ public class Ordering {
}
// --------------------------------------------------------------------------------------------
-
-
-
public Ordering clone() {
Ordering newOrdering = new Ordering();
@@ -223,7 +220,6 @@ public class Ordering {
newOrdering.orders.addAll(this.orders);
return newOrdering;
}
-
@Override
public int hashCode() {
@@ -235,7 +231,6 @@ public class Ordering {
return result;
}
-
@Override
public boolean equals(Object obj) {
if (this == obj) {
@@ -273,7 +268,7 @@ public class Ordering {
}
public String toString() {
- final StringBuffer buf = new StringBuffer("[");
+ final StringBuilder buf = new StringBuilder("[");
for (int i = 0; i < indexes.size(); i++) {
if (i != 0) {
buf.append(",");
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java
new file mode 100644
index 0000000..7810c3e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/DataExchangeMode.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+
+/**
+ * Defines how the data exchange between two specific operators happens.
+ */
+public enum DataExchangeMode {
+
+ /**
+ * The data exchange is streamed, sender and receiver are online at the same time,
+ * and the receiver back-pressures the sender.
+ */
+ PIPELINED,
+
+ /**
+ * The data exchange is decoupled. The sender first produces its entire result and finishes.
+ * After that, the receiver is stated and may consume the data.
+ */
+ BATCH,
+
+ /**
+ * The data exchange starts like ine {@link #PIPELINED} and falls back to {@link #BATCH}
+ * for recovery runs.
+ */
+ PIPELINE_WITH_BATCH_FALLBACK;
+
+ // ------------------------------------------------------------------------
+
+ public static DataExchangeMode getForForwardExchange(ExecutionMode mode) {
+ return FORWARD[mode.ordinal()];
+ }
+
+ public static DataExchangeMode getForShuffleOrBroadcast(ExecutionMode mode) {
+ return SHUFFLE[mode.ordinal()];
+ }
+
+ public static DataExchangeMode getPipelineBreakingExchange(ExecutionMode mode) {
+ return BREAKING[mode.ordinal()];
+ }
+
+ /**
+ * Computes the mode of data exchange to be used for a, given an execution mode and ship strategy.
+ * The type of the data exchange depends also on whether this connection has been identified to require
+ * pipeline breaking for deadlock avoidance.
+ * <ul>
+ * <li>If the connection is set to be pipeline breaking, this returns the pipeline breaking variant
+ * of the execution mode
+ * {@link org.apache.flink.runtime.io.network.DataExchangeMode#getPipelineBreakingExchange(org.apache.flink.api.common.ExecutionMode)}.
+ * </li>
+ * <li>If the data exchange is a simple FORWARD (one-to-one communication), this returns
+ * {@link org.apache.flink.runtime.io.network.DataExchangeMode#getForForwardExchange(org.apache.flink.api.common.ExecutionMode)}.
+ * </li>
+ * <li>If otherwise, this returns
+ * {@link org.apache.flink.runtime.io.network.DataExchangeMode#getForShuffleOrBroadcast(org.apache.flink.api.common.ExecutionMode)}.
+ * </li>
+ * </ul>
+ *
+ * @param shipStrategy The ship strategy (FORWARD, PARTITION, BROADCAST, ...) of the runtime data exchange.
+ * @return The data exchange mode for the connection, given the concrete ship strategy.
+ */
+ public static DataExchangeMode select(ExecutionMode executionMode, ShipStrategyType shipStrategy,
+ boolean breakPipeline) {
+
+ if (shipStrategy == null || shipStrategy == ShipStrategyType.NONE) {
+ throw new IllegalArgumentException("shipStrategy may not be null or NONE");
+ }
+ if (executionMode == null) {
+ throw new IllegalArgumentException("executionMode may not mbe null");
+ }
+
+ if (breakPipeline) {
+ return getPipelineBreakingExchange(executionMode);
+ }
+ else if (shipStrategy == ShipStrategyType.FORWARD) {
+ return getForForwardExchange(executionMode);
+ }
+ else {
+ return getForShuffleOrBroadcast(executionMode);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ private static final DataExchangeMode[] FORWARD = new DataExchangeMode[ExecutionMode.values().length];
+
+ private static final DataExchangeMode[] SHUFFLE = new DataExchangeMode[ExecutionMode.values().length];
+
+ private static final DataExchangeMode[] BREAKING = new DataExchangeMode[ExecutionMode.values().length];
+
+ // initialize the map between execution modes and exchange modes in
+ static {
+ FORWARD[ExecutionMode.PIPELINED_FORCED.ordinal()] = PIPELINED;
+ SHUFFLE[ExecutionMode.PIPELINED_FORCED.ordinal()] = PIPELINED;
+ BREAKING[ExecutionMode.PIPELINED_FORCED.ordinal()] = PIPELINED;
+
+ FORWARD[ExecutionMode.PIPELINED.ordinal()] = PIPELINED;
+ SHUFFLE[ExecutionMode.PIPELINED.ordinal()] = PIPELINED;
+ BREAKING[ExecutionMode.PIPELINED.ordinal()] = BATCH;
+
+ FORWARD[ExecutionMode.BATCH.ordinal()] = PIPELINED;
+ SHUFFLE[ExecutionMode.BATCH.ordinal()] = BATCH;
+ BREAKING[ExecutionMode.BATCH.ordinal()] = BATCH;
+
+ FORWARD[ExecutionMode.BATCH_FORCED.ordinal()] = BATCH;
+ SHUFFLE[ExecutionMode.BATCH_FORCED.ordinal()] = BATCH;
+ BREAKING[ExecutionMode.BATCH_FORCED.ordinal()] = BATCH;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/1c50d87c/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java
new file mode 100644
index 0000000..cae80e8
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network;
+
+import org.apache.flink.api.common.ExecutionMode;
+import org.junit.Test;
+
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * This test verifies that the data exchange modes are defined for every execution mode.
+ */
+public class DataExchangeModeTest {
+
+ @Test
+ public void testForward() {
+ for (ExecutionMode mode : ExecutionMode.values()) {
+ assertNotNull(DataExchangeMode.getForForwardExchange(mode));
+ }
+ }
+
+ @Test
+ public void testShuffleAndBroadcast() {
+ for (ExecutionMode mode : ExecutionMode.values()) {
+ assertNotNull(DataExchangeMode.getForShuffleOrBroadcast(mode));
+ }
+ }
+
+ @Test
+ public void testPipelineBreaking() {
+ for (ExecutionMode mode : ExecutionMode.values()) {
+ assertNotNull(DataExchangeMode.getPipelineBreakingExchange(mode));
+ }
+ }
+}