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));
+		}
+	}
+}