You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mx...@apache.org on 2015/03/23 09:09:21 UTC

[1/9] flink git commit: [FLINK-1679] deprecate API methods to set the parallelism

Repository: flink
Updated Branches:
  refs/heads/master 35f34162a -> 126f9f799


[FLINK-1679] deprecate API methods to set the parallelism


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d994d2e6
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d994d2e6
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d994d2e6

Branch: refs/heads/master
Commit: d994d2e6a0c37ed3156fdd670aca1bebb839983d
Parents: 35f3416
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 18 10:44:42 2015 +0100
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Mar 23 08:56:18 2015 +0100

----------------------------------------------------------------------
 .../flink/api/common/ExecutionConfig.java       | 39 ++++++++++++++-
 .../flink/api/common/operators/Operator.java    | 41 ++++++++++++---
 .../flink/api/java/CollectionEnvironment.java   | 11 ++++-
 .../flink/api/java/ExecutionEnvironment.java    | 43 ++++++++++++++--
 .../flink/api/scala/ExecutionEnvironment.scala  | 26 ++++++++--
 .../spargel/java/record/SpargelIteration.java   | 14 ++++--
 .../environment/StreamExecutionEnvironment.java | 52 ++++++++++++++++----
 .../api/scala/StreamExecutionEnvironment.scala  | 26 ++++++++--
 8 files changed, 217 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/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 aa025c1..a2df438 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -137,8 +137,25 @@ public class ExecutionConfig implements Serializable {
 	 *
 	 * @return The parallelism used by operations, unless they override that value. This method
 	 *         returns {@code -1}, if the environment's default parallelism should be used.
+	 * @deprecated Please use {@link #getParallelism}
 	 */
+	@Deprecated
 	public int getDegreeOfParallelism() {
+		return getParallelism();
+	}
+
+	/**
+	 * Gets the parallelism with which operation are executed by default. Operations can
+	 * individually override this value to use a specific parallelism.
+	 *
+	 * Other operations may need to run with a different parallelism - for example calling
+	 * a reduce operation over the entire data set will involve an operation that runs
+	 * with a parallelism of one (the final reduce to the single result value).
+	 *
+	 * @return The parallelism used by operations, unless they override that value. This method
+	 *         returns {@code -1}, if the environment's default parallelism should be used.
+	 */
+	public int getParallelism() {
 		return parallelism;
 	}
 
@@ -150,14 +167,32 @@ public class ExecutionConfig implements Serializable {
 	 * This method overrides the default parallelism for this environment.
 	 * The local execution environment uses by default a value equal to the number of hardware
 	 * contexts (CPU cores / threads). When executing the program via the command line client
-	 * from a JAR file, the default degree of parallelism is the one configured for that setup.
+	 * from a JAR file, the default parallelism is the one configured for that setup.
 	 *
 	 * @param parallelism The parallelism to use
+	 * @deprecated Please use {@link #setParallelism}
 	 */
+	@Deprecated
 	public ExecutionConfig setDegreeOfParallelism(int parallelism) {
+		return setParallelism(parallelism);
+	}
+
+	/**
+	 * Sets the parallelism for operations executed through this environment.
+	 * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with
+	 * x parallel instances.
+	 * <p>
+	 * This method overrides the default parallelism for this environment.
+	 * The local execution environment uses by default a value equal to the number of hardware
+	 * contexts (CPU cores / threads). When executing the program via the command line client
+	 * from a JAR file, the default parallelism is the one configured for that setup.
+	 *
+	 * @param parallelism The parallelism to use
+	 */
+	public ExecutionConfig setParallelism(int parallelism) {
 		if (parallelism < 1 && parallelism != -1) {
 			throw new IllegalArgumentException(
-					"Degree of parallelism must be at least one, or -1 (use system default).");
+					"Parallelism must be at least one, or -1 (use system default).");
 		}
 		this.parallelism = parallelism;
 		return this;

http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
index 85b352a..840c253 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
@@ -40,7 +40,7 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 	
 	protected String name;								// the name of the contract instance. optional.
 		
-	private int degreeOfParallelism = -1;				// the number of parallel instances to use. -1, if unknown
+	private int parallelism = -1;				// the number of parallel instances to use. -1, if unknown
 
 	/**
 	 * The return type of the user function.
@@ -160,25 +160,50 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 	}
 
 	/**
-	 * Gets the degree of parallelism for this contract instance. The degree of parallelism denotes
+	 * Gets the parallelism for this contract instance. The parallelism denotes
 	 * how many parallel instances of the user function will be spawned during the execution. If this
 	 * value is <code>-1</code>, then the system will decide the number of parallel instances by itself.
 	 * 
-	 * @return The degree of parallelism.
+	 * @return The parallelism.
+	 * @deprecated Please use {@link #getParallelism}
 	 */
+	@Deprecated
 	public int getDegreeOfParallelism() {
-		return this.degreeOfParallelism;
+		return getParallelism();
 	}
 
 	/**
-	 * Sets the degree of parallelism for this contract instance. The degree of parallelism denotes
+	 * Gets the parallelism for this contract instance. The parallelism denotes
+	 * how many parallel instances of the user function will be spawned during the execution. If this
+	 * value is <code>-1</code>, then the system will decide the number of parallel instances by itself.
+	 *
+	 * @return The parallelism.
+	 */
+	public int getParallelism() {
+		return this.parallelism;
+	}
+
+	/**
+	 * Sets the parallelism for this contract instance. The parallelism denotes
 	 * how many parallel instances of the user function will be spawned during the execution. Set this
 	 * value to <code>-1</code> to let the system decide on its own.
 	 * 
-	 * @param degree The number of parallel instances to spawn. -1, if unspecified.
+	 * @param parallelism The number of parallel instances to spawn. -1, if unspecified.
+	 * @deprecated Please use {@link #setParallelism}
+	 */
+	@Deprecated
+	public void setDegreeOfParallelism(int parallelism) {
+		setParallelism(parallelism);
+	}
+	/**
+	 * Sets the parallelism for this contract instance. The parallelism denotes
+	 * how many parallel instances of the user function will be spawned during the execution. Set this
+	 * value to <code>-1</code> to let the system decide on its own.
+	 *
+	 * @param parallelism The number of parallel instances to spawn. -1, if unspecified.
 	 */
-	public void setDegreeOfParallelism(int degree) {
-		this.degreeOfParallelism = degree;
+	public void setParallelism(int parallelism) {
+		this.parallelism = parallelism;
 	}
 	
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
index c61d624..b48debc 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
@@ -32,9 +32,18 @@ public class CollectionEnvironment extends ExecutionEnvironment {
 		CollectionExecutor exec = new CollectionExecutor(getConfig());
 		return exec.execute(p);
 	}
-	
+
+	/**
+	 * @deprecated Please use {@link #getParallelism}
+	 */
 	@Override
+	@Deprecated
 	public int getDegreeOfParallelism() {
+		return getParallelism();
+	}
+
+	@Override
+	public int getParallelism() {
 		return 1; // always serial
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index 929bf6e..aac3147 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -142,9 +142,26 @@ public abstract class ExecutionEnvironment {
 	 * 
 	 * @return The degree of parallelism used by operations, unless they override that value. This method
 	 *         returns {@code -1}, if the environments default parallelism should be used.
+	 * @deprecated Please use {@link #getParallelism}
 	 */
+	@Deprecated
 	public int getDegreeOfParallelism() {
-		return config.getDegreeOfParallelism();
+		return getParallelism();
+	}
+
+	/**
+	 * Gets the parallelism with which operation are executed by default. Operations can
+	 * individually override this value to use a specific parallelism via
+	 * {@link Operator#setParallelism(int)}. Other operations may need to run with a different
+	 * parallelism - for example calling
+	 * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire
+	 * set will insert eventually an operation that runs non-parallel (parallelism of one).
+	 *
+	 * @return The parallelism used by operations, unless they override that value. This method
+	 *         returns {@code -1}, if the environments default parallelism should be used.
+	 */
+	public int getParallelism() {
+		return config.getParallelism();
 	}
 	
 	/**
@@ -157,10 +174,28 @@ public abstract class ExecutionEnvironment {
 	 * contexts (CPU cores / threads). When executing the program via the command line client 
 	 * from a JAR file, the default degree of parallelism is the one configured for that setup.
 	 * 
-	 * @param degreeOfParallelism The degree of parallelism
+	 * @param parallelism The parallelism
+	 * @deprecated Please use {@link #setParallelism}
+	 */
+	@Deprecated
+	public void setDegreeOfParallelism(int parallelism) {
+		setParallelism(parallelism);
+	}
+
+	/**
+	 * Sets the parallelism for operations executed through this environment.
+	 * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with
+	 * x parallel instances.
+	 * <p>
+	 * This method overrides the default parallelism for this environment.
+	 * The {@link LocalEnvironment} uses by default a value equal to the number of hardware
+	 * contexts (CPU cores / threads). When executing the program via the command line client
+	 * from a JAR file, the default parallelism is the one configured for that setup.
+	 *
+	 * @param parallelism The parallelism
 	 */
-	public void setDegreeOfParallelism(int degreeOfParallelism) {
-		config.setDegreeOfParallelism(degreeOfParallelism);
+	public void setParallelism(int parallelism) {
+		config.setParallelism(parallelism);
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
index f49aef1..cccea78 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
@@ -82,17 +82,35 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
    * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with
    * x parallel instances. This value can be overridden by specific operations using
    * [[DataSet.setParallelism]].
+   * @deprecated Please use [[setParallelism]]
    */
-  def setDegreeOfParallelism(degreeOfParallelism: Int): Unit = {
-    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+  @deprecated
+  def setDegreeOfParallelism(parallelism: Int): Unit = {
+    setParallelism(parallelism)
   }
 
   /**
    * Returns the default degree of parallelism for this execution environment. Note that this
    * value can be overridden by individual operations using [[DataSet.setParallelism]
    */
-  def getDegreeOfParallelism = javaEnv.getDegreeOfParallelism
-  
+  def setParallelism(parallelism: Int): Unit = {
+    javaEnv.setParallelism(parallelism)
+  }
+
+  /**
+   * Returns the default parallelism for this execution environment. Note that this
+   * value can be overridden by individual operations using [[DataSet.setParallelism]]
+   * @deprecated Please use [[getParallelism]]
+   */
+  @deprecated
+  def getDegreeOfParallelism = javaEnv.getParallelism
+
+  /**
+   * Returns the default parallelism for this execution environment. Note that this
+   * value can be overridden by individual operations using [[DataSet.setParallelism]]
+   */
+  def getParallelism = javaEnv.getParallelism
+
   /**
    * Sets the number of times that failed tasks are re-executed. A value of zero
    * effectively disables fault tolerance. A value of "-1" indicates that the system

http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
index f647e5d..8f1839d 100644
--- a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
+++ b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
@@ -127,9 +127,17 @@ public class SpargelIteration {
 	public Operator<?> getOutput() {
 		return this.iteration;
 	}
-	
-	public void setDegreeOfParallelism(int dop) {
-		this.iteration.setDegreeOfParallelism(dop);
+
+	/**
+	 * @deprecated Please use {@link #setParallelism}
+	 */
+	@Deprecated
+	public void setDegreeOfParallelism(int parallelism) {
+		setParallelism(parallelism);
+	}
+
+	public void setParallelism(int parallelism) {
+		this.iteration.setParallelism(parallelism);
 	}
 	
 	public void setNumberOfIterations(int iterations) {

http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 994ff15..dc1826b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -97,16 +97,30 @@ public abstract class StreamExecutionEnvironment {
 	 * default. Operations can individually override this value to use a
 	 * specific degree of parallelism.
 	 * 
-	 * @return The degree of parallelism used by operations, unless they
+	 * @return The parallelism used by operations, unless they
 	 *         override that value.
+	 * @deprecated Please use {@link #getParallelism}
 	 */
+	@Deprecated
 	public int getDegreeOfParallelism() {
-		return config.getDegreeOfParallelism();
+		return getParallelism();
 	}
 
 	/**
-	 * Sets the degree of parallelism (DOP) for operations executed through this
-	 * environment. Setting a DOP of x here will cause all operators (such as
+	 * Gets the parallelism with which operation are executed by
+	 * default. Operations can individually override this value to use a
+	 * specific parallelism.
+	 *
+	 * @return The parallelism used by operations, unless they
+	 *         override that value.
+	 */
+	public int getParallelism() {
+		return config.getParallelism();
+	}
+
+	/**
+	 * Sets the parallelism for operations executed through this
+	 * environment. Setting a parallelism of x here will cause all operators (such as
 	 * map, batchReduce) to run with x parallel instances. This method overrides
 	 * the default parallelism for this environment. The
 	 * {@link LocalStreamEnvironment} uses by default a value equal to the
@@ -114,14 +128,32 @@ public abstract class StreamExecutionEnvironment {
 	 * program via the command line client from a JAR file, the default degree
 	 * of parallelism is the one configured for that setup.
 	 * 
-	 * @param degreeOfParallelism
-	 *            The degree of parallelism
+	 * @param parallelism The parallelism
+	 * @deprecated Please use {@link #setParallelism}
+	 */
+	@Deprecated
+	public StreamExecutionEnvironment setDegreeOfParallelism(int parallelism) {
+		return setParallelism(parallelism);
+	}
+
+	/**
+	 * Sets the parallelism for operations executed through this
+	 * environment. Setting a parallelism of x here will cause all operators (such as
+	 * map, batchReduce) to run with x parallel instances. This method overrides
+	 * the default parallelism for this environment. The
+	 * {@link LocalStreamEnvironment} uses by default a value equal to the
+	 * number of hardware contexts (CPU cores / threads). When executing the
+	 * program via the command line client from a JAR file, the default degree
+	 * of parallelism is the one configured for that setup.
+	 *
+	 * @param parallelism
+	 *            The parallelism
 	 */
-	public StreamExecutionEnvironment setDegreeOfParallelism(int degreeOfParallelism) {
-		if (degreeOfParallelism < 1) {
-			throw new IllegalArgumentException("Degree of parallelism must be at least one.");
+	public StreamExecutionEnvironment setParallelism(int parallelism) {
+		if (parallelism < 1) {
+			throw new IllegalArgumentException("parallelism must be at least one.");
 		}
-		config.setDegreeOfParallelism(degreeOfParallelism);
+		config.setParallelism(parallelism);
 		return this;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d994d2e6/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 2208388..598b590 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -37,16 +37,36 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
    * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with
    * x parallel instances. This value can be overridden by specific operations using
    * [[DataStream.setParallelism]].
+   * @deprecated Please use [[setParallelism]]
    */
+  @deprecated
   def setDegreeOfParallelism(degreeOfParallelism: Int): Unit = {
-    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+    javaEnv.setParallelism(degreeOfParallelism)
+  }
+
+  /**
+   * Sets the parallelism for operations executed through this environment.
+   * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
+   * with x parallel instances. This value can be overridden by specific operations using
+   * [[DataStream.setParallelism]].
+   */
+  def setParallelism(parallelism: Int): Unit = {
+    javaEnv.setParallelism(parallelism)
   }
 
   /**
-   * Returns the default degree of parallelism for this execution environment. Note that this
+   * Returns the default parallelism for this execution environment. Note that this
+   * value can be overridden by individual operations using [[DataStream.setParallelism]]
+   * @deprecated Please use [[getParallelism]]
+   */
+  @deprecated
+  def getDegreeOfParallelism = javaEnv.getParallelism
+
+  /**
+   * Returns the default parallelism for this execution environment. Note that this
    * value can be overridden by individual operations using [[DataStream.setParallelism]]
    */
-  def getDegreeOfParallelism = javaEnv.getDegreeOfParallelism
+  def getParallelism = javaEnv.getParallelism
 
   /**
    * Sets the maximum time frequency (milliseconds) for the flushing of the


[3/9] flink git commit: [FLINK-1679] use a consistent name for parallelism

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java
index 8a324a3..8197e27 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MatchJoinCancelingITCase.java
@@ -37,10 +37,10 @@ import org.apache.flink.util.Collector;
 
 @SuppressWarnings("deprecation")
 public class MatchJoinCancelingITCase extends CancellingTestBase {
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
 	public MatchJoinCancelingITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	// --------------- Test Sort Matches that are canceled while still reading / sorting -----------------
@@ -60,7 +60,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 3000, 10*1000);
 	}
@@ -81,7 +81,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 5000, 10*1000);
 	}
@@ -102,7 +102,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 5000);
 		
@@ -129,7 +129,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 30 * 1000, 30 * 1000);
 	}
@@ -157,7 +157,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 10 * 1000, 20 * 1000);
 	}
@@ -183,7 +183,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 		GenericDataSink sink = new GenericDataSink(new DiscardingOutputFormat(), matcher, "Sink");
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 10 * 1000, 10 * 1000);
 	}
@@ -191,7 +191,7 @@ public class MatchJoinCancelingITCase extends CancellingTestBase {
 	// -------------------------------------- Test System corner cases ---------------------------------
 	
 //	@Test
-	public void testCancelSortMatchWithHighDOP() throws Exception {
+	public void testCancelSortMatchWithHighparallelism() throws Exception {
 		
 		GenericDataSource<InfiniteIntegerInputFormat> source1 =
 			new GenericDataSource<InfiniteIntegerInputFormat>(new InfiniteIntegerInputFormat(), "Source 1");

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
index 53b70ad..62e2893 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
@@ -30,7 +30,7 @@ import org.junit.Test;
 
 public class LocalExecutorITCase {
 
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
 	@Test
 	public void testLocalExecutorWithWordCount() {
@@ -50,11 +50,11 @@ public class LocalExecutorITCase {
 
 			LocalExecutor executor = new LocalExecutor();
 			executor.setDefaultOverwriteFiles(true);
-			executor.setTaskManagerNumSlots(DOP);
+			executor.setTaskManagerNumSlots(parallelism);
 			executor.setPrintStatusDuringExecution(false);
 			executor.start();
 			
-			executor.executePlan(wc.getPlan(Integer.valueOf(DOP).toString(), inFile.toURI().toString(),
+			executor.executePlan(wc.getPlan(Integer.valueOf(parallelism).toString(), inFile.toURI().toString(),
 					outFile.toURI().toString()));
 			executor.stop();
 		} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
index a69b5c2..5c1c500 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
@@ -28,7 +28,7 @@ public class WordCountITCase extends JavaProgramTestBase {
 	protected String resultPath;
 
 	public WordCountITCase(){
-		setDegreeOfParallelism(4);
+		setParallelism(4);
 		setNumTaskManagers(2);
 		setTaskManagerNumSlots(2);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java
index 9021c6a..0d2c469 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java
@@ -42,7 +42,7 @@ import org.apache.flink.util.Collector;
 @SuppressWarnings("deprecation")
 public class TaskFailureITCase extends FailingTestBase {
 
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
 	// input for map tasks
 	private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" +
@@ -57,7 +57,7 @@ public class TaskFailureITCase extends FailingTestBase {
 	private String resultPath;
 
 	public TaskFailureITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	@Override
@@ -85,7 +85,7 @@ public class TaskFailureITCase extends FailingTestBase {
 
 		// generate plan
 		Plan plan = new Plan(output);
-		plan.setDefaultParallelism(DOP);
+		plan.setDefaultParallelism(parallelism);
 
 		// optimize and compile plan 
 		Optimizer pc = new Optimizer(new DataStatistics());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
index 0a4673a..9023a1f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
@@ -58,7 +58,7 @@ public class CoGroupConnectedComponentsITCase extends RecordAPITestBase {
 	protected String resultPath;
 
 	public CoGroupConnectedComponentsITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	
@@ -71,7 +71,7 @@ public class CoGroupConnectedComponentsITCase extends RecordAPITestBase {
 	
 	@Override
 	protected Plan getTestJob() {
-		return getPlan(DOP, verticesPath, edgesPath, resultPath, 100);
+		return getPlan(parallelism, verticesPath, edgesPath, resultPath, 100);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
index f6ada63..df3c00d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
@@ -41,7 +41,7 @@ public class ConnectedComponentsITCase extends RecordAPITestBase {
 	protected String resultPath;
 
 	public ConnectedComponentsITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	
@@ -55,7 +55,7 @@ public class ConnectedComponentsITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WorksetConnectedComponents cc = new WorksetConnectedComponents();
-		return cc.getPlan(Integer.valueOf(DOP).toString(),  verticesPath, edgesPath, resultPath, "100");
+		return cc.getPlan(Integer.valueOf(parallelism).toString(),  verticesPath, edgesPath, resultPath, "100");
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
index 0b1e372..d5d150d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
@@ -65,7 +65,7 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends RecordAPITestBa
 	
 	public ConnectedComponentsWithDeferredUpdateITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	@Override
@@ -78,7 +78,7 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends RecordAPITestBa
 	@Override
 	protected Plan getTestJob() {
 		boolean extraMapper = config.getBoolean("ExtraMapper", false);
-		return getPlan(DOP, verticesPath, edgesPath, resultPath, 100, extraMapper);
+		return getPlan(parallelism, verticesPath, edgesPath, resultPath, 100, extraMapper);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
index 33650c5..b97d0ad 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
@@ -60,7 +60,7 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends RecordAPITest
 	protected String resultPath;
 
 	public ConnectedComponentsWithSolutionSetFirstITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	
@@ -73,7 +73,7 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends RecordAPITest
 	
 	@Override
 	protected Plan getTestJob() {
-		return getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput(DOP, verticesPath, edgesPath,
+		return getPlanForWorksetConnectedComponentsWithSolutionSetAsFirstInput(parallelism, verticesPath, edgesPath,
 				resultPath, 100);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
index 39617b4..e2d095d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
@@ -40,7 +40,7 @@ public class DanglingPageRankITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		DanglingPageRank pr = new DanglingPageRank();
 		Plan plan = pr.getPlan(
-			String.valueOf(DOP),
+			String.valueOf(parallelism),
 			pagesPath,
 			edgesPath,
 			resultPath,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
index 7f5015e..caa9d37 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
@@ -40,7 +40,7 @@ public class DeltaIterationNotDependingOnSolutionSetITCase extends JavaProgramTe
 	protected void testProgram() throws Exception {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(1);
+			env.setParallelism(1);
 
 			DataSet<Tuple2<Long, Long>> input = env.generateSequence(0, 9).map(new Duplicator<Long>());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java
index cf59a3f..bf459c6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaPageRankITCase.java
@@ -42,7 +42,7 @@ public class DeltaPageRankITCase extends RecordAPITestBase {
 	
 	@Override
 	protected Plan getTestJob() {
-		String[] params = { String.valueOf(DOP) , verticesPath, edgesPath, resultPath, "3" };
+		String[] params = { String.valueOf(parallelism) , verticesPath, edgesPath, resultPath, "3" };
 		
 		WorksetConnectedComponents cc = new WorksetConnectedComponents();
 		return cc.getPlan(params);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
index 15079ec..0635fe5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
@@ -43,7 +43,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 	
 	private static final int MAX_ITERATIONS = 20;
-	private static final int DOP = 1;
+	private static final int parallelism = 1;
 
 	protected static List<Tuple2<Long, Long>> verticesInput = new ArrayList<Tuple2<Long, Long>>();
 	protected static List<Tuple2<Long, Long>> edgesInput = new ArrayList<Tuple2<Long, Long>>();
@@ -51,7 +51,7 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 	private String expectedResult;
 
 	public DependencyConnectedComponentsITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	
@@ -113,7 +113,7 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 		public static String runProgram(String resultPath) throws Exception {
 			
 			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DOP);
+			env.setParallelism(parallelism);
 			
 			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
 			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
index 7b6cf11..0915a42 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
@@ -47,7 +47,7 @@ public class IterationTerminationWithTerminationTail extends RecordAPITestBase {
 	protected String resultPath;
 
 	public IterationTerminationWithTerminationTail(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	@Override
@@ -63,7 +63,7 @@ public class IterationTerminationWithTerminationTail extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		return getTestPlanPlan(DOP, dataPath, resultPath);
+		return getTestPlanPlan(parallelism, dataPath, resultPath);
 	}
 	
 	private static Plan getTestPlanPlan(int numSubTasks, String input, String output) {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
index f05b1c2..3ce021b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
@@ -47,7 +47,7 @@ public class IterationTerminationWithTwoTails extends RecordAPITestBase {
 	protected String resultPath;
 
 	public IterationTerminationWithTwoTails(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -63,7 +63,7 @@ public class IterationTerminationWithTwoTails extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		return getTestPlanPlan(DOP, dataPath, resultPath);
+		return getTestPlanPlan(parallelism, dataPath, resultPath);
 	}
 	
 	private static Plan getTestPlanPlan(int numSubTasks, String input, String output) {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
index 3116a9d..cb16c15 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
@@ -61,7 +61,7 @@ public class IterationWithAllReducerITCase extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		Plan plan = getTestPlanPlan(DOP, dataPath, resultPath);
+		Plan plan = getTestPlanPlan(parallelism, dataPath, resultPath);
 		return plan;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
index f626b75..c11c9ea 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
@@ -52,7 +52,7 @@ public class IterationWithChainingITCase extends RecordAPITestBase {
 
 	public IterationWithChainingITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -74,7 +74,7 @@ public class IterationWithChainingITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
-		config1.setInteger("ChainedMapperITCase#NoSubtasks", DOP);
+		config1.setInteger("ChainedMapperITCase#NoSubtasks", parallelism);
 		return toParameterList(config1);
 	}
 
@@ -103,7 +103,7 @@ public class IterationWithChainingITCase extends RecordAPITestBase {
 	static Plan getTestPlan(int numSubTasks, String input, String output) {
 
 		FileDataSource initialInput = new FileDataSource(new PointInFormat(), input, "Input");
-		initialInput.setDegreeOfParallelism(1);
+		initialInput.setParallelism(1);
 
 		BulkIteration iteration = new BulkIteration("Loop");
 		iteration.setInput(initialInput);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java
index 4cd72fd..ac3659a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterativeKMeansITCase.java
@@ -35,7 +35,7 @@ public class IterativeKMeansITCase extends RecordAPITestBase {
 	protected String resultPath;
 
 	public IterativeKMeansITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	@Override
@@ -48,7 +48,7 @@ public class IterativeKMeansITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		KMeansBroadcast kmi = new KMeansBroadcast();
-		return kmi.getPlan(String.valueOf(DOP), dataPath, clusterPath, resultPath, "20");
+		return kmi.getPlan(String.valueOf(parallelism), dataPath, clusterPath, resultPath, "20");
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java
index 2491028..fcf43df 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansITCase.java
@@ -35,7 +35,7 @@ public class KMeansITCase extends RecordAPITestBase {
 	protected String resultPath;
 
 	public KMeansITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -48,7 +48,7 @@ public class KMeansITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		KMeansBroadcast kmi = new KMeansBroadcast();
-		return kmi.getPlan(String.valueOf(DOP), dataPath, clusterPath, resultPath, "20");
+		return kmi.getPlan(String.valueOf(parallelism), dataPath, clusterPath, resultPath, "20");
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
index 450b360..fa13656 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/LineRankITCase.java
@@ -59,7 +59,7 @@
 //
 //	public LineRankITCase(Configuration config) {
 //		super(config);
-//		setTaskManagerNumSlots(DOP);
+//		setTaskManagerNumSlots(parallelism);
 //	}
 //
 //	@Override
@@ -85,7 +85,7 @@
 //	@Parameters
 //	public static Collection<Object[]> getConfigurations() {
 //		Configuration config1 = new Configuration();
-//		config1.setInteger("NumSubtasks", DOP);
+//		config1.setInteger("NumSubtasks", parallelism);
 //		config1.setInteger("NumIterations", 5);
 //		return toParameterList(config1);
 //	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java
index 8e42dd7..946d89b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java
@@ -43,7 +43,7 @@ public class PageRankITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		SimplePageRank pr = new SimplePageRank();
 		Plan plan = pr.getPlan(
-			String.valueOf(DOP), 
+			String.valueOf(parallelism),
 			pagesPath,
 			edgesPath,
 			resultPath,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 5aa6b42..44544d3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
@@ -51,7 +51,7 @@ import org.junit.runners.Parameterized;
 public class AggregatorsITCase extends MultipleProgramsTestBase {
 
 	private static final int MAX_ITERATIONS = 20;
-	private static final int DOP = 2;
+	private static final int parallelism = 2;
 	private static final String NEGATIVE_ELEMENTS_AGGR = "count.negative.elements";
 
 	public AggregatorsITCase(TestExecutionMode mode){
@@ -81,7 +81,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DOP);
+		env.setParallelism(parallelism);
 
 		DataSet<Integer> initialSolutionSet = CollectionDataSets.getIntegerDataSet(env);
 		IterativeDataSet<Integer> iteration = initialSolutionSet.iterate(MAX_ITERATIONS);
@@ -110,7 +110,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DOP);
+		env.setParallelism(parallelism);
 
 		DataSet<Integer> initialSolutionSet = CollectionDataSets.getIntegerDataSet(env);
 		IterativeDataSet<Integer> iteration = initialSolutionSet.iterate(MAX_ITERATIONS);
@@ -139,7 +139,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DOP);
+		env.setParallelism(parallelism);
 
 		DataSet<Integer> initialSolutionSet = CollectionDataSets.getIntegerDataSet(env);
 		IterativeDataSet<Integer> iteration = initialSolutionSet.iterate(MAX_ITERATIONS);
@@ -168,7 +168,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DOP);
+		env.setParallelism(parallelism);
 
 		DataSet<Tuple2<Integer, Integer>> initialSolutionSet = CollectionDataSets.getIntegerDataSet(env).map(new TupleMakerMap());
 
@@ -202,7 +202,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DOP);
+		env.setParallelism(parallelism);
 
 		DataSet<Tuple2<Integer, Integer>> initialSolutionSet = CollectionDataSets.getIntegerDataSet(env).map(new TupleMakerMap());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
index faaa541..8bf50de 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
@@ -41,7 +41,7 @@ import org.apache.flink.api.java.operators.IterativeDataSet;
 public class ConnectedComponentsWithParametrizableAggregatorITCase extends JavaProgramTestBase {
 
 	private static final int MAX_ITERATIONS = 5;
-	private static final int DOP = 1;
+	private static final int parallelism = 1;
 
 	protected static List<Tuple2<Long, Long>> verticesInput = new ArrayList<Tuple2<Long, Long>>();
 	protected static List<Tuple2<Long, Long>> edgesInput = new ArrayList<Tuple2<Long, Long>>();
@@ -118,7 +118,7 @@ public class ConnectedComponentsWithParametrizableAggregatorITCase extends JavaP
 		public static String runProgram(String resultPath) throws Exception {
 
 			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DOP);
+			env.setParallelism(parallelism);
 
 			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
 			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
index 4d890e9..e616a2b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
@@ -44,7 +44,7 @@ import org.apache.flink.api.java.operators.IterativeDataSet;
 public class ConnectedComponentsWithParametrizableConvergenceITCase extends JavaProgramTestBase {
 
 	private static final int MAX_ITERATIONS = 10;
-	private static final int DOP = 1;
+	private static final int parallelism = 1;
 
 	protected static List<Tuple2<Long, Long>> verticesInput = new ArrayList<Tuple2<Long, Long>>();
 	protected static List<Tuple2<Long, Long>> edgesInput = new ArrayList<Tuple2<Long, Long>>();
@@ -111,7 +111,7 @@ public class ConnectedComponentsWithParametrizableConvergenceITCase extends Java
 		public static String runProgram(String resultPath) throws Exception {
 
 			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DOP);
+			env.setParallelism(parallelism);
 
 			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
 			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
index 0e568b6..7cec122 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
@@ -89,9 +89,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	private static final long MEM_PER_CONSUMER = 3;
 
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
-	private static final double MEM_FRAC_PER_CONSUMER = (double)MEM_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*DOP;
+	private static final double MEM_FRAC_PER_CONSUMER = (double)MEM_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*parallelism;
 
 	protected String verticesPath;
 
@@ -101,7 +101,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	public ConnectedComponentsNepheleITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Parameters
@@ -135,14 +135,14 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		int type = config.getInteger("testcase", 0);
 		switch (type) {
 		case 1:
-			return createJobGraphUnifiedTails(verticesPath, edgesPath, resultPath, DOP, maxIterations);
+			return createJobGraphUnifiedTails(verticesPath, edgesPath, resultPath, parallelism, maxIterations);
 		case 2:
-			return createJobGraphSeparateTails(verticesPath, edgesPath, resultPath, DOP, maxIterations);
+			return createJobGraphSeparateTails(verticesPath, edgesPath, resultPath, parallelism, maxIterations);
 		case 3:
-			return createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(verticesPath, edgesPath, resultPath, DOP,
+			return createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(verticesPath, edgesPath, resultPath, parallelism,
 				maxIterations);
 		case 4:
-			return createJobGraphSolutionSetUpdateAndWorksetTail(verticesPath, edgesPath, resultPath, DOP,
+			return createJobGraphSolutionSetUpdateAndWorksetTail(verticesPath, edgesPath, resultPath, parallelism,
 				maxIterations);
 		default:
 			throw new RuntimeException("Broken test configuration");

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java
index f22bc84..516309c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java
@@ -40,7 +40,7 @@ public class DanglingPageRankNepheleITCase extends RecordAPITestBase {
 	protected String resultPath;
 
 	public DanglingPageRankNepheleITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	
@@ -54,7 +54,7 @@ public class DanglingPageRankNepheleITCase extends RecordAPITestBase {
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
 		String[] parameters = new String[] {
-			Integer.valueOf(DOP).toString(),
+			Integer.valueOf(parallelism).toString(),
 			pagesWithRankPath,
 			edgesPath,
 			resultPath,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
index c4ffd02..ba22ce5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
@@ -29,7 +29,7 @@ public class DanglingPageRankWithCombinerNepheleITCase extends RecordAPITestBase
 	protected String resultPath;
 
 	public DanglingPageRankWithCombinerNepheleITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	@Override
@@ -42,7 +42,7 @@ public class DanglingPageRankWithCombinerNepheleITCase extends RecordAPITestBase
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
 		String[] parameters = new String[] {
-			Integer.valueOf(DOP).toString(),
+			Integer.valueOf(parallelism).toString(),
 			pagesWithRankPath,
 			edgesPath,
 			resultPath,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
index 2a8e84d..69ff083 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
@@ -79,7 +79,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 
 	public IterationWithChainingNepheleITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -99,7 +99,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 	@Parameterized.Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("ChainedMapperNepheleITCase#NoSubtasks", DOP);
+		config.setInteger("ChainedMapperNepheleITCase#NoSubtasks", parallelism);
 		config.setInteger("ChainedMapperNepheleITCase#MaxIterations", 2);
 		return toParameterList(config);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
index 58969c9..153a85e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
@@ -40,20 +40,20 @@ public class JobGraphUtils {
 	private JobGraphUtils() {}
 	
 	public static <T extends FileInputFormat<?>> InputFormatVertex createInput(T stub, String path, String name, JobGraph graph,
-			int degreeOfParallelism)
+			int parallelism)
 	{
 		stub.setFilePath(path);
-		return createInput(new UserCodeObjectWrapper<T>(stub), name, graph, degreeOfParallelism);
+		return createInput(new UserCodeObjectWrapper<T>(stub), name, graph, parallelism);
 	}
 
 	private static <T extends InputFormat<?,?>> InputFormatVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
-			int degreeOfParallelism)
+			int parallelism)
 	{
 		InputFormatVertex inputVertex = new InputFormatVertex(name);
 		graph.addVertex(inputVertex);
 		
 		inputVertex.setInvokableClass(DataSourceTask.class);
-		inputVertex.setParallelism(degreeOfParallelism);
+		inputVertex.setParallelism(parallelism);
 
 		TaskConfig inputConfig = new TaskConfig(inputVertex.getConfiguration());
 		inputConfig.setStubWrapper(stub);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
index 0705500..8801dd6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
@@ -96,7 +96,7 @@ public class CustomCompensatableDanglingPageRank {
 		
 	public static JobGraph getJobGraph(String[] args) throws Exception {
 
-		int degreeOfParallelism = 2;
+		int parallelism = 2;
 		String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
 		String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR +
 //			"test-inputs/danglingpagerank/adjacencylists";
@@ -114,7 +114,7 @@ public class CustomCompensatableDanglingPageRank {
 		double messageLoss = 0.75;
 
 		if (args.length >= 14) {
-			degreeOfParallelism = Integer.parseInt(args[0]);
+			parallelism = Integer.parseInt(args[0]);
 			pageWithRankInputPath = args[1];
 			adjacencyListInputPath = args[2];
 			outputPath = args[3];
@@ -138,7 +138,7 @@ public class CustomCompensatableDanglingPageRank {
 
 		// page rank input
 		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
+			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0);
@@ -147,7 +147,7 @@ public class CustomCompensatableDanglingPageRank {
 
 		// edges as adjacency list
 		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
+			adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer);
@@ -155,7 +155,7 @@ public class CustomCompensatableDanglingPageRank {
 
 		// --------------- the head ---------------------
 		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			degreeOfParallelism);
+			parallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
 		
@@ -200,7 +200,7 @@ public class CustomCompensatableDanglingPageRank {
 		// --------------- the join ---------------------
 		
 		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, degreeOfParallelism);
+			"IterationIntermediate", jobGraph, parallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
 //		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
@@ -228,7 +228,7 @@ public class CustomCompensatableDanglingPageRank {
 		// ---------------- the tail (co group) --------------------
 		
 		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			degreeOfParallelism);
+			parallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
 		tailConfig.setIsWorksetUpdate();
@@ -264,7 +264,7 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -273,7 +273,7 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 
-		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
@@ -292,7 +292,7 @@ public class CustomCompensatableDanglingPageRank {
 		JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE);
 		JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
+		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism);
 
 		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
index 7bc300f..6f19c03 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
@@ -97,7 +97,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 	public static JobGraph getJobGraph(String[] args) throws Exception {
 
-		int degreeOfParallelism = 2;
+		int parallelism = 2;
 		String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
 		String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR +
 //			"test-inputs/danglingpagerank/adjacencylists";
@@ -114,7 +114,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		double messageLoss = 0.75;
 
 		if (args.length >= 14) {
-			degreeOfParallelism = Integer.parseInt(args[0]);
+			parallelism = Integer.parseInt(args[0]);
 			pageWithRankInputPath = args[1];
 			adjacencyListInputPath = args[2];
 			outputPath = args[3];
@@ -138,7 +138,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		// page rank input
 		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
+			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0);
@@ -147,7 +147,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		// edges as adjacency list
 		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
+			adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer);
@@ -155,7 +155,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		// --------------- the head ---------------------
 		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			degreeOfParallelism);
+			parallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
 		
@@ -200,7 +200,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		// --------------- the join ---------------------
 		
 		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, degreeOfParallelism);
+			"IterationIntermediate", jobGraph, parallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
 //		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
@@ -241,7 +241,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		// ---------------- the tail (co group) --------------------
 		
 		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			degreeOfParallelism);
+			parallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
 		tailConfig.setIsWorksetUpdate();
@@ -278,7 +278,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the output ---------------------
 
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -287,7 +287,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the auxiliaries ---------------------
 
-		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
@@ -306,7 +306,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE);
 		JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
+		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism);
 
 		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
index f06f723..8216ccb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
@@ -77,7 +77,7 @@ public class CompensatableDanglingPageRank {
 		
 	public static JobGraph getJobGraph(String[] args) throws Exception {
 
-		int degreeOfParallelism = 2;
+		int parallelism = 2;
 		String pageWithRankInputPath = ""; // "file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
 		String adjacencyListInputPath = ""; // "file://" + PlayConstants.PLAY_DIR +
 //			"test-inputs/danglingpagerank/adjacencylists";
@@ -95,7 +95,7 @@ public class CompensatableDanglingPageRank {
 		double messageLoss = 0.75;
 
 		if (args.length >= 15) {
-			degreeOfParallelism = Integer.parseInt(args[0]);
+			parallelism = Integer.parseInt(args[0]);
 			pageWithRankInputPath = args[1];
 			adjacencyListInputPath = args[2];
 			outputPath = args[3];
@@ -119,7 +119,7 @@ public class CompensatableDanglingPageRank {
 
 		// page rank input
 		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
+			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		pageWithRankInputConfig.setOutputComparator(fieldZeroComparator, 0);
@@ -128,14 +128,14 @@ public class CompensatableDanglingPageRank {
 
 		// edges as adjacency list
 		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
+			adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
 		adjacencyListInputConfig.setOutputSerializer(recSerializer);
 		adjacencyListInputConfig.setOutputComparator(fieldZeroComparator, 0);
 
 		// --------------- the head ---------------------
-		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, degreeOfParallelism);
+		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, parallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
 		
@@ -179,7 +179,7 @@ public class CompensatableDanglingPageRank {
 
 		// --------------- the join ---------------------
 		
-		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, "IterationIntermediate", jobGraph, degreeOfParallelism);
+		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, "IterationIntermediate", jobGraph, parallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
 //		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
@@ -207,7 +207,7 @@ public class CompensatableDanglingPageRank {
 		// ---------------- the tail (co group) --------------------
 		
 		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			degreeOfParallelism);
+			parallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
 		tailConfig.setIsWorksetUpdate();
@@ -244,7 +244,7 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(recSerializer, 0);
@@ -253,7 +253,7 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 
-		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
@@ -272,7 +272,7 @@ public class CompensatableDanglingPageRank {
 		JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE);
 		JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL);
 		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
+		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism);
 
 		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 dbf4798..5dc3867 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java
@@ -61,7 +61,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testIntSortingDOP1() throws Exception {
+	public void testIntSortingParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -76,7 +76,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testStringSortingDOP1() throws Exception {
+	public void testStringSortingParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -98,7 +98,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testTupleSortingSingleAscDOP1() throws Exception {
+	public void testTupleSortingSingleAscParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -134,7 +134,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testTupleSortingSingleDescDOP1() throws Exception {
+	public void testTupleSortingSingleDescParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -169,7 +169,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testTupleSortingDualDOP1() throws Exception {
+	public void testTupleSortingDualParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -204,7 +204,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testTupleSortingNestedDOP1() throws Exception {
+	public void testTupleSortingNestedParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -230,7 +230,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testTupleSortingNestedDOP1_2() throws Exception {
+	public void testTupleSortingNestedParallelism1_2() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -256,7 +256,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testPojoSortingSingleDOP1() throws Exception {
+	public void testPojoSortingSingleParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -279,7 +279,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testPojoSortingDualDOP1() throws Exception {
+	public void testPojoSortingDualParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -306,7 +306,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testPojoSortingNestedDOP1() throws Exception {
+	public void testPojoSortingNestedParallelism1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -333,7 +333,7 @@ public class DataSinkITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testSortingDOP4() throws Exception {
+	public void testSortingParallelism4() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java
index 494f354..2a97c60 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java
@@ -321,12 +321,12 @@ public class GroupCombineITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	// check if dop 1 results in the same data like a shuffle
+	// check if parallelism of 1 results in the same data like a shuffle
 	public void testCheckPartitionShuffleDOP1() throws Exception {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		// data
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 e5f91b4..c5067f9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
@@ -138,7 +138,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
@@ -346,7 +346,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(2); // important because it determines how often the combiner is called
+		env.setParallelism(2); // important because it determines how often the combiner is called
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Tuple2<Integer, String>> reduceDs = ds.
@@ -394,7 +394,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * check correctness of groupReduce with descending group sort
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
@@ -456,7 +456,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 	 	 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
@@ -590,7 +590,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * check correctness of groupReduce with descending group sort
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
@@ -613,7 +613,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Test int-based definition on group sort, for (full) nested Tuple
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
 		DataSet<String> reduceDs = ds.groupBy("f1").sortGroup(0, Order.DESCENDING).reduceGroup(new NestedTupleReducer());
@@ -631,7 +631,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Test int-based definition on group sort, for (partial) nested Tuple ASC
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
 		// f0.f0 is first integer
@@ -653,7 +653,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Test string-based definition on group sort, for (partial) nested Tuple DESC
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
 		// f0.f0 is first integer
@@ -672,7 +672,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Test string-based definition on group sort, for two grouping keys
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
 		// f0.f0 is first integer
@@ -691,7 +691,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Test string-based definition on group sort, for two grouping keys with Pojos
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<PojoContainingTupleAndWritable> ds = CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env);
 		// f0.f0 is first integer
@@ -711,7 +711,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
@@ -830,7 +830,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Tuple2<Integer, String>> reduceDs = ds.
@@ -870,7 +870,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
 		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds
@@ -915,7 +915,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Test grouping with pojo containing multiple pojos (was a bug)
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<CollectionDataSets.PojoWithMultiplePojos> ds = CollectionDataSets.getPojoWithMultiplePojos(env);
 
@@ -947,7 +947,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Test Java collections within pojos ( == test kryo)
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<CollectionDataSets.PojoWithCollection> ds = CollectionDataSets.getPojoWithCollection(env);
 		// f0.f0 is first integer
@@ -982,7 +982,7 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
 		 * Group by generic type
 		 */
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<CollectionDataSets.PojoWithCollection> ds = CollectionDataSets.getPojoWithCollection(env);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 375baee..0080fb1 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
@@ -527,7 +527,7 @@ public class JoinITCase extends MultipleProgramsTestBase {
 				ds1.join(ds2).where("nestedPojo.longNumber", "number", "str").equalTo("f6","f0","f1");
 
 		joinDs.writeAsCsv(resultPath);
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 		env.execute();
 
 		expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
@@ -548,7 +548,7 @@ public class JoinITCase extends MultipleProgramsTestBase {
 				ds1.join(ds2).where("nestedPojo.longNumber", "number","nestedTupleWithCustom.f0").equalTo("f6","f0","f2");
 
 		joinDs.writeAsCsv(resultPath);
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 		env.execute();
 
 		expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
@@ -569,7 +569,7 @@ public class JoinITCase extends MultipleProgramsTestBase {
 				ds1.join(ds2).where("nestedTupleWithCustom.f0","nestedTupleWithCustom.f1.myInt","nestedTupleWithCustom.f1.myLong").equalTo("f2","f3","f4");
 
 		joinDs.writeAsCsv(resultPath);
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 		env.execute();
 
 		expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
@@ -590,7 +590,7 @@ public class JoinITCase extends MultipleProgramsTestBase {
 				ds1.join(ds2).where(0).equalTo("f0.f0", "f0.f1"); // key is now Tuple2<Integer, Integer>
 
 		joinDs.writeAsCsv(resultPath);
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 		env.execute();
 
 		expected = "((1,1),one),((1,1),one)\n" +
@@ -612,7 +612,7 @@ public class JoinITCase extends MultipleProgramsTestBase {
 				ds1.join(ds2).where("f0.f0").equalTo("f0.f0"); // key is now Integer from Tuple2<Integer, Integer>
 
 		joinDs.writeAsCsv(resultPath);
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 		env.execute();
 
 		expected = "((1,1),one),((1,1),one)\n" +
@@ -633,7 +633,7 @@ public class JoinITCase extends MultipleProgramsTestBase {
 				ds1.join(ds2).where("*").equalTo("*");
 
 		joinDs.writeAsCsv(resultPath);
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 		env.execute();
 
 		expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n"+
@@ -655,7 +655,7 @@ public class JoinITCase extends MultipleProgramsTestBase {
 				ds2.join(ds2).where("f1.f0").equalTo("f0.f0");
 
 		joinDs.writeAsCsv(resultPath);
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 		env.execute();
 
 		expected = "((1,1,Hi),(1,1,Hi)),((1,1,Hi),(1,1,Hi))\n" +

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 e1603ca..3637680 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java
@@ -148,8 +148,8 @@ public class PartitionITCase extends MultipleProgramsTestBase {
 		env.execute();
 
 		StringBuilder result = new StringBuilder();
-		int numPerPartition = 2220 / env.getDegreeOfParallelism() / 10;
-		for (int i = 0; i < env.getDegreeOfParallelism(); i++) {
+		int numPerPartition = 2220 / env.getParallelism() / 10;
+		for (int i = 0; i < env.getParallelism(); i++) {
 			result.append('(').append(i).append(',').append(numPerPartition).append(")\n");
 		}
 
@@ -190,13 +190,13 @@ public class PartitionITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testHashPartitionByKeyFieldAndDifferentDOP() throws Exception {
+	public void testHashPartitionByKeyFieldAndDifferentParallelism() throws Exception {
 		/*
-		 * Test hash partition by key field and different DOP
+		 * Test hash partition by key field and different parallelism
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(3);
+		env.setParallelism(3);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		DataSet<Long> uniqLongs = ds
@@ -221,7 +221,7 @@ public class PartitionITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(3);
+		env.setParallelism(3);
 
 		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
 		DataSet<Long> uniqLongs = ds

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 4bba558..d961f3a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java
@@ -72,7 +72,7 @@ public class SortPartitionITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(4);
+		env.setParallelism(4);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		ds
@@ -94,7 +94,7 @@ public class SortPartitionITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(2);
+		env.setParallelism(2);
 
 		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
 		ds
@@ -117,7 +117,7 @@ public class SortPartitionITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(4);
+		env.setParallelism(4);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		ds
@@ -139,7 +139,7 @@ public class SortPartitionITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(2);
+		env.setParallelism(2);
 
 		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
 		ds
@@ -162,7 +162,7 @@ public class SortPartitionITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(3);
+		env.setParallelism(3);
 
 		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
 		ds
@@ -185,7 +185,7 @@ public class SortPartitionITCase extends MultipleProgramsTestBase {
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(3);
+		env.setParallelism(3);
 
 		DataSet<POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
 		ds
@@ -202,17 +202,17 @@ public class SortPartitionITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void testSortPartitionDOPChange() throws Exception {
+	public void testSortPartitionParallelismChange() throws Exception {
 		/*
-		 * Test sort partition with DOP change
+		 * Test sort partition with parallelism change
 		 */
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(3);
+		env.setParallelism(3);
 
 		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
 		ds
-				.sortPartition(1, Order.DESCENDING).setParallelism(3) // change DOP
+				.sortPartition(1, Order.DESCENDING).setParallelism(3) // change parallelism
 				.mapPartition(new OrderCheckMapper<Tuple3<Integer, Long, String>>(new Tuple3Checker()))
 				.distinct()
 				.writeAsText(resultPath);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
index 8ddd7bc..aeab77b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
@@ -78,7 +78,7 @@ public class AutoParallelismITCase {
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 					"localhost", cluster.getJobManagerRPCPort());
 
-			env.setDegreeOfParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
+			env.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
 
 			DataSet<Integer> result = env
 					.createInput(new ParallelismDependentInputFormat())

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
index c308007..39a08d2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
@@ -34,7 +34,7 @@ public class CustomPartitioningITCase extends JavaProgramTestBase {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
 		if (!isCollectionExecution()) {
-			Assert.assertTrue(env.getDegreeOfParallelism() > 1);
+			Assert.assertTrue(env.getParallelism() > 1);
 		}
 		
 		env.generateSequence(1, 1000)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java
index 2087b63..e0ebadd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/NullValuesITCase.java
@@ -48,7 +48,7 @@ public class NullValuesITCase {
 			ExecutionEnvironment env =
 					ExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getJobManagerRPCPort());
 
-			env.setDegreeOfParallelism(1);
+			env.setParallelism(1);
 
 			DataSet<String> data = env.fromElements("hallo")
 					.map(new MapFunction<String, String>() {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java
index 220611d..be05186 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java
@@ -146,23 +146,23 @@ public class CoGroupITCase extends RecordAPITestBase {
 		FileDataSource input_left =  new FileDataSource(new CoGroupTestInFormat(), leftInPath);
 		DelimitedInputFormat.configureDelimitedFormat(input_left)
 			.recordDelimiter('\n');
-		input_left.setDegreeOfParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1));
+		input_left.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1));
 
 		FileDataSource input_right =  new FileDataSource(new CoGroupTestInFormat(), rightInPath);
 		DelimitedInputFormat.configureDelimitedFormat(input_right)
 			.recordDelimiter('\n');
-		input_right.setDegreeOfParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1));
+		input_right.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1));
 
 		CoGroupOperator testCoGrouper = CoGroupOperator.builder(new TestCoGrouper(), StringValue.class, 0, 0)
 			.build();
-		testCoGrouper.setDegreeOfParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1));
+		testCoGrouper.setParallelism(config.getInteger("CoGroupTest#NoSubtasks", 1));
 		testCoGrouper.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY,
 				config.getString("CoGroupTest#LocalStrategy", ""));
 		testCoGrouper.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY,
 				config.getString("CoGroupTest#ShipStrategy", ""));
 
 		FileDataSink output = new FileDataSink(new CoGroupOutFormat(), resultPath);
-		output.setDegreeOfParallelism(1);
+		output.setParallelism(1);
 
 		output.setInput(testCoGrouper);
 		testCoGrouper.setFirstInput(input_left);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java
index f6b4127..3fde5a9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java
@@ -122,16 +122,16 @@ public class CrossITCase extends RecordAPITestBase {
 				new ContractITCaseInputFormat(), leftInPath);
 		DelimitedInputFormat.configureDelimitedFormat(input_left)
 			.recordDelimiter('\n');
-		input_left.setDegreeOfParallelism(config.getInteger("CrossTest#NoSubtasks", 1));
+		input_left.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1));
 
 		FileDataSource input_right = new FileDataSource(
 				new ContractITCaseInputFormat(), rightInPath);
 		DelimitedInputFormat.configureDelimitedFormat(input_right)
 			.recordDelimiter('\n');
-		input_right.setDegreeOfParallelism(config.getInteger("CrossTest#NoSubtasks", 1));
+		input_right.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1));
 
 		CrossOperator testCross = CrossOperator.builder(new TestCross()).build();
-		testCross.setDegreeOfParallelism(config.getInteger("CrossTest#NoSubtasks", 1));
+		testCross.setParallelism(config.getInteger("CrossTest#NoSubtasks", 1));
 		testCross.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY,
 				config.getString("CrossTest#LocalStrategy", ""));
 		if (config.getString("CrossTest#ShipStrategy", "").equals("BROADCAST_FIRST")) {
@@ -151,7 +151,7 @@ public class CrossITCase extends RecordAPITestBase {
 
 		FileDataSink output = new FileDataSink(
 				new ContractITCaseOutputFormat(), resultPath);
-		output.setDegreeOfParallelism(1);
+		output.setParallelism(1);
 
 		output.setInput(testCross);
 		testCross.setFirstInput(input_left);


[6/9] flink git commit: [FLINK-1679] use a consistent name for parallelism

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
index 47446dd..55cbb0f 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FlatMapOperator.java
@@ -57,13 +57,13 @@ public class FlatMapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, Fl
 		FlatMapOperatorBase<IN, OUT, FlatMapFunction<IN, OUT>> po = new FlatMapOperatorBase<IN, OUT, FlatMapFunction<IN, OUT>>(function, new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType()), name);
 		// set input
 		po.setInput(input);
-		// set dop
+		// set parallelism
 		if(this.getParallelism() > 0) {
-			// use specified dop
-			po.setDegreeOfParallelism(this.getParallelism());
+			// use specified parallelism
+			po.setParallelism(this.getParallelism());
 		} else {
-			// if no dop has been specified, use dop of input operator to enable chaining
-			po.setDegreeOfParallelism(input.getDegreeOfParallelism());
+			// if no parallelism has been specified, use parallelism of input operator to enable chaining
+			po.setParallelism(input.getParallelism());
 		}
 		
 		return po;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java
index 617162b..3c1d47c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java
@@ -103,8 +103,8 @@ public class GroupCombineOperator<IN, OUT> extends SingleInputUdfOperator<IN, OU
 					new GroupCombineOperatorBase<IN, OUT, FlatCombineFunction<IN, OUT>>(function, operatorInfo, new int[0], name);
 
 			po.setInput(input);
-			// the degree of parallelism for a non grouped reduce can only be 1
-			po.setDegreeOfParallelism(1);
+			// the parallelism for a non grouped reduce can only be 1
+			po.setParallelism(1);
 			return po;
 		}
 
@@ -130,13 +130,13 @@ public class GroupCombineOperator<IN, OUT> extends SingleInputUdfOperator<IN, OU
 				}
 				po.setGroupOrder(o);
 
-				po.setDegreeOfParallelism(this.getParallelism());
+				po.setParallelism(this.getParallelism());
 				return po;
 			} else {
 				PlanUnwrappingGroupCombineOperator<IN, OUT, ?> po = translateSelectorFunctionReducer(
 						selectorKeys, function, getInputType(), getResultType(), name, input);
 
-				po.setDegreeOfParallelism(this.getParallelism());
+				po.setParallelism(this.getParallelism());
 				return po;
 			}
 		}
@@ -148,7 +148,7 @@ public class GroupCombineOperator<IN, OUT> extends SingleInputUdfOperator<IN, OU
 					new GroupCombineOperatorBase<IN, OUT, FlatCombineFunction<IN, OUT>>(function, operatorInfo, logicalKeyPositions, name);
 
 			po.setInput(input);
-			po.setDegreeOfParallelism(getParallelism());
+			po.setParallelism(getParallelism());
 
 			// set group order
 			if (grouper instanceof SortedGrouping) {
@@ -193,7 +193,7 @@ public class GroupCombineOperator<IN, OUT> extends SingleInputUdfOperator<IN, OU
 		mapper.setInput(input);
 
 		// set the mapper's parallelism to the input parallelism to make sure it is chained
-		mapper.setDegreeOfParallelism(input.getDegreeOfParallelism());
+		mapper.setParallelism(input.getParallelism());
 
 		return reducer;
 	}
@@ -220,7 +220,7 @@ public class GroupCombineOperator<IN, OUT> extends SingleInputUdfOperator<IN, OU
 		mapper.setInput(input);
 
 		// set the mapper's parallelism to the input parallelism to make sure it is chained
-		mapper.setDegreeOfParallelism(input.getDegreeOfParallelism());
+		mapper.setParallelism(input.getParallelism());
 
 		return reducer;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
index c542192..e809623 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java
@@ -138,8 +138,8 @@ public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 			
 			po.setCombinable(combinable);
 			po.setInput(input);
-			// the degree of parallelism for a non grouped reduce can only be 1
-			po.setDegreeOfParallelism(1);
+			// the parallelism for a non grouped reduce can only be 1
+			po.setParallelism(1);
 			return po;
 		}
 	
@@ -165,14 +165,14 @@ public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 				}
 				po.setGroupOrder(o);
 
-				po.setDegreeOfParallelism(this.getParallelism());
+				po.setParallelism(this.getParallelism());
 				po.setCustomPartitioner(grouper.getCustomPartitioner());
 				return po;
 			} else {
 				PlanUnwrappingReduceGroupOperator<IN, OUT, ?> po = translateSelectorFunctionReducer(
 							selectorKeys, function, getInputType(), getResultType(), name, input, isCombinable());
 
-				po.setDegreeOfParallelism(this.getParallelism());
+				po.setParallelism(this.getParallelism());
 				po.setCustomPartitioner(grouper.getCustomPartitioner());
 				return po;
 			}
@@ -186,7 +186,7 @@ public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 
 			po.setCombinable(combinable);
 			po.setInput(input);
-			po.setDegreeOfParallelism(getParallelism());
+			po.setParallelism(getParallelism());
 			po.setCustomPartitioner(grouper.getCustomPartitioner());
 			
 			// set group order
@@ -233,7 +233,7 @@ public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 		mapper.setInput(input);
 		
 		// set the mapper's parallelism to the input parallelism to make sure it is chained
-		mapper.setDegreeOfParallelism(input.getDegreeOfParallelism());
+		mapper.setParallelism(input.getParallelism());
 		
 		return reducer;
 	}
@@ -261,7 +261,7 @@ public class GroupReduceOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
 		mapper.setInput(input);
 
 		// set the mapper's parallelism to the input parallelism to make sure it is chained
-		mapper.setDegreeOfParallelism(input.getDegreeOfParallelism());
+		mapper.setParallelism(input.getParallelism());
 
 		return reducer;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
index 8b61779..e450ae1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
@@ -273,8 +273,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 						translateSelectorFunctionJoin(selectorKeys1, selectorKeys2, function, 
 						getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
 				
-				// set dop
-				po.setDegreeOfParallelism(this.getParallelism());
+				// set parallelism
+				po.setParallelism(this.getParallelism());
 				
 				translated = po;
 			}
@@ -292,8 +292,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 								function, getInput1Type(), getInput2Type(), getResultType(), name,
 								input1, input2);
 
-				// set dop
-				po.setDegreeOfParallelism(this.getParallelism());
+				// set parallelism
+				po.setParallelism(this.getParallelism());
 
 				translated = po;
 			}
@@ -311,8 +311,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 						translateSelectorFunctionJoinLeft(selectorKeys1, logicalKeyPositions2, function,
 								getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
 
-				// set dop
-				po.setDegreeOfParallelism(this.getParallelism());
+				// set parallelism
+				po.setParallelism(this.getParallelism());
 
 				translated = po;
 			}
@@ -332,8 +332,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 				// set inputs
 				po.setFirstInput(input1);
 				po.setSecondInput(input2);
-				// set dop
-				po.setDegreeOfParallelism(this.getParallelism());
+				// set parallelism
+				po.setParallelism(this.getParallelism());
 				
 				translated = po;
 			}
@@ -375,9 +375,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			
 			keyMapper1.setInput(input1);
 			keyMapper2.setInput(input2);
-			// set dop
-			keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
-			keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
+			// set parallelism
+			keyMapper1.setParallelism(input1.getParallelism());
+			keyMapper2.setParallelism(input2.getParallelism());
 			
 			return join;
 		}
@@ -427,8 +427,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			join.setSecondInput(keyMapper2);
 			
 			keyMapper2.setInput(input2);
-			// set dop
-			keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
+			// set parallelism
+			keyMapper2.setParallelism(input2.getParallelism());
 			
 			return join;
 		}
@@ -477,8 +477,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			join.setSecondInput(input2);
 			
 			keyMapper1.setInput(input1);
-			// set dop
-			keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
+			// set parallelism
+			keyMapper1.setParallelism(input1.getParallelism());
 
 			return join;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
index 7d2bbaa..2663a2a 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapOperator.java
@@ -61,13 +61,13 @@ public class MapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, MapOpe
 				new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType()), name);
 		// set input
 		po.setInput(input);
-		// set dop
+		// set parallelism
 		if(this.getParallelism() > 0) {
-			// use specified dop
-			po.setDegreeOfParallelism(this.getParallelism());
+			// use specified parallelism
+			po.setParallelism(this.getParallelism());
 		} else {
-			// if no dop has been specified, use dop of input operator to enable chaining
-			po.setDegreeOfParallelism(input.getDegreeOfParallelism());
+			// if no parallelism has been specified, use parallelism of input operator to enable chaining
+			po.setParallelism(input.getParallelism());
 		}
 		
 		return po;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
index a6c69c1..d8a1abd 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/MapPartitionOperator.java
@@ -60,13 +60,13 @@ public class MapPartitionOperator<IN, OUT> extends SingleInputUdfOperator<IN, OU
 		MapPartitionOperatorBase<IN, OUT, MapPartitionFunction<IN, OUT>> po = new MapPartitionOperatorBase<IN, OUT, MapPartitionFunction<IN, OUT>>(function, new UnaryOperatorInformation<IN, OUT>(getInputType(), getResultType()), name);
 		// set input
 		po.setInput(input);
-		// set dop
+		// set parallelism
 		if(this.getParallelism() > 0) {
-			// use specified dop
-			po.setDegreeOfParallelism(this.getParallelism());
+			// use specified parallelism
+			po.setParallelism(this.getParallelism());
 		} else {
-			// if no dop has been specified, use dop of input operator to enable chaining
-			po.setDegreeOfParallelism(input.getDegreeOfParallelism());
+			// if no parallelism has been specified, use parallelism of input operator to enable chaining
+			po.setParallelism(input.getParallelism());
 		}
 		
 		return po;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
index 0f8a3eb..6d02749 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java
@@ -32,7 +32,7 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 
 	protected String name;
 	
-	protected int dop = -1;
+	protected int parallelism = -1;
 
 	protected Operator(ExecutionEnvironment context, TypeInformation<OUT> resultType) {
 		super(context, resultType);
@@ -58,12 +58,12 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 	}
 	
 	/**
-	 * Returns the degree of parallelism of this operator.
+	 * Returns the parallelism of this operator.
 	 * 
-	 * @return The degree of parallelism of this operator.
+	 * @return The parallelism of this operator.
 	 */
 	public int getParallelism() {
-		return this.dop;
+		return this.parallelism;
 	}
 
 	/**
@@ -82,17 +82,17 @@ public abstract class Operator<OUT, O extends Operator<OUT, O>> extends DataSet<
 	}
 	
 	/**
-	 * Sets the degree of parallelism for this operator.
+	 * Sets the parallelism for this operator.
 	 * The degree must be 1 or more.
 	 * 
-	 * @param dop The degree of parallelism for this operator.
-	 * @return The operator with set degree of parallelism.
+	 * @param parallelism The parallelism for this operator.
+	 * @return The operator with set parallelism.
 	 */
-	public O setParallelism(int dop) {
-		if(dop < 1) {
+	public O setParallelism(int parallelism) {
+		if(parallelism < 1) {
 			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
 		}
-		this.dop = dop;
+		this.parallelism = parallelism;
 		
 		@SuppressWarnings("unchecked")
 		O returnType = (O) this;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index 68a216b..28c1c29 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -218,7 +218,7 @@ public class OperatorTranslation {
 		iterationOperator.setMaximumNumberOfIterations(iterationEnd.getMaxIterations());
 		
 		if (iterationHead.getParallelism() > 0) {
-			iterationOperator.setDegreeOfParallelism(iterationHead.getParallelism());
+			iterationOperator.setParallelism(iterationHead.getParallelism());
 		}
 
 		DeltaIteration.SolutionSetPlaceHolder<D> solutionSetPlaceHolder = iterationHead.getSolutionSet();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
index edb5a68..bf9c8e8 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
@@ -118,7 +118,7 @@ public class PartitionOperator<T> extends SingleInputOperator<T, T, PartitionOpe
 			PartitionOperatorBase<T> noop = new PartitionOperatorBase<T>(operatorInfo, pMethod, name);
 			
 			noop.setInput(input);
-			noop.setDegreeOfParallelism(getParallelism());
+			noop.setParallelism(getParallelism());
 			
 			return noop;
 		} 
@@ -131,7 +131,7 @@ public class PartitionOperator<T> extends SingleInputOperator<T, T, PartitionOpe
 				PartitionOperatorBase<T> noop = new PartitionOperatorBase<T>(operatorInfo, pMethod, logicalKeyPositions, name);
 				
 				noop.setInput(input);
-				noop.setDegreeOfParallelism(getParallelism());
+				noop.setParallelism(getParallelism());
 				noop.setCustomPartitioner(customPartitioner);
 				
 				return noop;
@@ -177,10 +177,10 @@ public class PartitionOperator<T> extends SingleInputOperator<T, T, PartitionOpe
 		
 		noop.setCustomPartitioner(customPartitioner);
 		
-		// set dop
-		keyExtractingMap.setDegreeOfParallelism(input.getDegreeOfParallelism());
-		noop.setDegreeOfParallelism(partitionDop);
-		keyRemovingMap.setDegreeOfParallelism(partitionDop);
+		// set parallelism
+		keyExtractingMap.setParallelism(input.getParallelism());
+		noop.setParallelism(partitionDop);
+		keyRemovingMap.setParallelism(partitionDop);
 		
 		return keyRemovingMap;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java
index 16d9ff3..9b7d567 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java
@@ -71,8 +71,8 @@ public class ProjectOperator<IN, OUT extends Tuple>
 		PlanProjectOperator<IN, OUT> ppo = new PlanProjectOperator<IN, OUT>(fields, name, getInputType(), getResultType(), context.getConfig());
 		// set input
 		ppo.setInput(input);
-		// set dop
-		ppo.setDegreeOfParallelism(this.getParallelism());
+		// set parallelism
+		ppo.setParallelism(this.getParallelism());
 		ppo.setSemanticProperties(SemanticPropUtil.createProjectionPropertiesSingle(fields, (CompositeType<?>) getInputType()));
 
 		return ppo;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java
index d1ad4c3..5951df8 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ReduceOperator.java
@@ -90,8 +90,8 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
 					new ReduceOperatorBase<IN, ReduceFunction<IN>>(function, operatorInfo, new int[0], name);
 			
 			po.setInput(input);
-			// the degree of parallelism for a non grouped reduce can only be 1
-			po.setDegreeOfParallelism(1);
+			// the parallelism for a non grouped reduce can only be 1
+			po.setParallelism(1);
 			
 			return po;
 		}
@@ -118,7 +118,7 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
 			po.setCustomPartitioner(grouper.getCustomPartitioner());
 			
 			po.setInput(input);
-			po.setDegreeOfParallelism(getParallelism());
+			po.setParallelism(getParallelism());
 			
 			return po;
 		}
@@ -130,7 +130,7 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
 	// --------------------------------------------------------------------------------------------
 	
 	private static <T, K> MapOperatorBase<Tuple2<K, T>, T, ?> translateSelectorFunctionReducer(Keys.SelectorFunctionKeys<T, ?> rawKeys,
-			ReduceFunction<T> function, TypeInformation<T> inputType, String name, Operator<T> input, int dop)
+			ReduceFunction<T> function, TypeInformation<T> inputType, String name, Operator<T> input, int parallelism)
 	{
 		@SuppressWarnings("unchecked")
 		final Keys.SelectorFunctionKeys<T, K> keys = (Keys.SelectorFunctionKeys<T, K>) rawKeys;
@@ -148,10 +148,10 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
 		reducer.setInput(keyExtractingMap);
 		keyRemovingMap.setInput(reducer);
 		
-		// set dop
-		keyExtractingMap.setDegreeOfParallelism(input.getDegreeOfParallelism());
-		reducer.setDegreeOfParallelism(dop);
-		keyRemovingMap.setDegreeOfParallelism(dop);
+		// set parallelism
+		keyExtractingMap.setParallelism(input.getParallelism());
+		reducer.setParallelism(parallelism);
+		keyRemovingMap.setParallelism(parallelism);
 		
 		return keyRemovingMap;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java
index f6e3c2a..35c564b 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java
@@ -47,8 +47,8 @@ public class TypeSerializerFormatTest extends SequentialFormatTestBase<Tuple2<In
 
 	private BlockInfo block;
 
-	public TypeSerializerFormatTest(int numberOfTuples, long blockSize, int degreeOfParallelism) {
-		super(numberOfTuples, blockSize, degreeOfParallelism);
+	public TypeSerializerFormatTest(int numberOfTuples, long blockSize, int parallelism) {
+		super(numberOfTuples, blockSize, parallelism);
 
         resultType = TypeExtractor.getForObject(getRecord(0));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java
index 08eefff..63b4052 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java
@@ -39,8 +39,8 @@ public class AggregateTranslationTest {
 	@Test
 	public void translateAggregate() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 			
 			@SuppressWarnings("unchecked")
 			DataSet<Tuple3<Double, StringValue, Long>> initialData = 
@@ -58,7 +58,7 @@ public class AggregateTranslationTest {
 			assertEquals(1, reducer.getKeyColumns(0).length);
 			assertEquals(0, reducer.getKeyColumns(0)[0]);
 			
-			assertEquals(-1, reducer.getDegreeOfParallelism());
+			assertEquals(-1, reducer.getParallelism());
 			assertTrue(reducer.isCombinable());
 			
 			assertTrue(reducer.getInput() instanceof GenericDataSourceBase<?, ?>);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
index 55a2aff..ae89780 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java
@@ -59,14 +59,14 @@ public class DeltaIterationTranslationTest implements java.io.Serializable {
 			final int[] ITERATION_KEYS = new int[] {2};
 			final int NUM_ITERATIONS = 13;
 			
-			final int DEFAULT_DOP= 133;
-			final int ITERATION_DOP = 77;
+			final int DEFAULT_parallelism= 133;
+			final int ITERATION_parallelism = 77;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 			
 			// ------------ construct the test program ------------------
 			{
-				env.setDegreeOfParallelism(DEFAULT_DOP);
+				env.setParallelism(DEFAULT_parallelism);
 				
 				@SuppressWarnings("unchecked")
 				DataSet<Tuple3<Double, Long, String>> initialSolutionSet = env.fromElements(new Tuple3<Double, Long, String>(3.44, 5L, "abc"));
@@ -75,7 +75,7 @@ public class DeltaIterationTranslationTest implements java.io.Serializable {
 				DataSet<Tuple2<Double, String>> initialWorkSet = env.fromElements(new Tuple2<Double, String>(1.23, "abc"));
 				
 				DeltaIteration<Tuple3<Double, Long, String>, Tuple2<Double, String>> iteration = initialSolutionSet.iterateDelta(initialWorkSet, NUM_ITERATIONS, ITERATION_KEYS);
-				iteration.name(ITERATION_NAME).parallelism(ITERATION_DOP);
+				iteration.name(ITERATION_NAME).parallelism(ITERATION_parallelism);
 				
 				iteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
 				
@@ -100,7 +100,7 @@ public class DeltaIterationTranslationTest implements java.io.Serializable {
 			
 			// ------------- validate the plan ----------------
 			assertEquals(JOB_NAME, p.getJobName());
-			assertEquals(DEFAULT_DOP, p.getDefaultParallelism());
+			assertEquals(DEFAULT_parallelism, p.getDefaultParallelism());
 			
 			// validate the iteration
 			GenericDataSinkBase<?> sink1, sink2;
@@ -118,7 +118,7 @@ public class DeltaIterationTranslationTest implements java.io.Serializable {
 			// check the basic iteration properties
 			assertEquals(NUM_ITERATIONS, iteration.getMaximumNumberOfIterations());
 			assertArrayEquals(ITERATION_KEYS, iteration.getSolutionSetKeyFields());
-			assertEquals(ITERATION_DOP, iteration.getDegreeOfParallelism());
+			assertEquals(ITERATION_parallelism, iteration.getParallelism());
 			assertEquals(ITERATION_NAME, iteration.getName());
 			
 			MapOperatorBase<?, ?, ?> nextWorksetMapper = (MapOperatorBase<?, ?, ?>) iteration.getNextWorkset();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
index 0cf2ee2..b7fbb78 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
@@ -76,8 +76,8 @@ public class DistinctTranslationTest {
 	@Test
 	public void translateDistinctPlain() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 
 			DataSet<Tuple3<Double, StringValue, LongValue>> initialData = getSourceDataSet(env);
 
@@ -97,8 +97,8 @@ public class DistinctTranslationTest {
 			// check keys
 			assertArrayEquals(new int[] {0, 1, 2}, reducer.getKeyColumns(0));
 
-			// DOP was not configured on the operator
-			assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1);
+			// parallelism was not configured on the operator
+			assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1);
 
 			assertTrue(reducer.getInput() instanceof GenericDataSourceBase<?, ?>);
 		}
@@ -112,8 +112,8 @@ public class DistinctTranslationTest {
 	@Test
 	public void translateDistinctPlain2() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 
 			DataSet<CustomType> initialData = getSourcePojoDataSet(env);
 
@@ -133,8 +133,8 @@ public class DistinctTranslationTest {
 			// check keys
 			assertArrayEquals(new int[] {0}, reducer.getKeyColumns(0));
 
-			// DOP was not configured on the operator
-			assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1);
+			// parallelism was not configured on the operator
+			assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1);
 
 			assertTrue(reducer.getInput() instanceof GenericDataSourceBase<?, ?>);
 		}
@@ -148,8 +148,8 @@ public class DistinctTranslationTest {
 	@Test
 	public void translateDistinctPosition() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 
 			DataSet<Tuple3<Double, StringValue, LongValue>> initialData = getSourceDataSet(env);
 
@@ -169,8 +169,8 @@ public class DistinctTranslationTest {
 			// check keys
 			assertArrayEquals(new int[] {1, 2}, reducer.getKeyColumns(0));
 
-			// DOP was not configured on the operator
-			assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1);
+			// parallelism was not configured on the operator
+			assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1);
 
 			assertTrue(reducer.getInput() instanceof GenericDataSourceBase<?, ?>);
 		}
@@ -184,8 +184,8 @@ public class DistinctTranslationTest {
 	@Test
 	public void translateDistinctKeySelector() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 
 			DataSet<Tuple3<Double, StringValue, LongValue>> initialData = getSourceDataSet(env);
 
@@ -202,9 +202,9 @@ public class DistinctTranslationTest {
 			PlanUnwrappingReduceGroupOperator<?, ?, ?> reducer = (PlanUnwrappingReduceGroupOperator<?, ?, ?>) sink.getInput();
 			MapOperatorBase<?, ?, ?> keyExtractor = (MapOperatorBase<?, ?, ?>) reducer.getInput();
 
-			// check the DOPs
-			assertEquals(1, keyExtractor.getDegreeOfParallelism());
-			assertEquals(4, reducer.getDegreeOfParallelism());
+			// check the parallelisms
+			assertEquals(1, keyExtractor.getParallelism());
+			assertEquals(4, reducer.getParallelism());
 
 			// check types
 			TypeInformation<?> keyValueInfo = new TupleTypeInfo<Tuple2<StringValue, Tuple3<Double,StringValue,LongValue>>>(
@@ -232,8 +232,8 @@ public class DistinctTranslationTest {
 	@Test
 	public void translateDistinctExpressionKey() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 
 			DataSet<CustomType> initialData = getSourcePojoDataSet(env);
 
@@ -253,8 +253,8 @@ public class DistinctTranslationTest {
 			// check keys
 			assertArrayEquals(new int[] {0}, reducer.getKeyColumns(0));
 
-			// DOP was not configured on the operator
-			assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1);
+			// parallelism was not configured on the operator
+			assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1);
 
 			assertTrue(reducer.getInput() instanceof GenericDataSourceBase<?, ?>);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
index f9253f8..b578eb7 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java
@@ -45,8 +45,8 @@ public class ReduceTranslationTests implements java.io.Serializable {
 	@Test
 	public void translateNonGroupedReduce() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 			
 			DataSet<Tuple3<Double, StringValue, LongValue>> initialData = getSourceDataSet(env);
 			
@@ -69,8 +69,8 @@ public class ReduceTranslationTests implements java.io.Serializable {
 			// check keys
 			assertTrue(reducer.getKeyColumns(0) == null || reducer.getKeyColumns(0).length == 0);
 			
-			// DOP was not configured on the operator
-			assertTrue(reducer.getDegreeOfParallelism() == 1 || reducer.getDegreeOfParallelism() == -1);
+			// parallelism was not configured on the operator
+			assertTrue(reducer.getParallelism() == 1 || reducer.getParallelism() == -1);
 			
 			assertTrue(reducer.getInput() instanceof GenericDataSourceBase<?, ?>);
 		}
@@ -84,8 +84,8 @@ public class ReduceTranslationTests implements java.io.Serializable {
 	@Test
 	public void translateGroupedReduceNoMapper() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 			
 			DataSet<Tuple3<Double, StringValue, LongValue>> initialData = getSourceDataSet(env);
 			
@@ -108,8 +108,8 @@ public class ReduceTranslationTests implements java.io.Serializable {
 			assertEquals(initialData.getType(), reducer.getOperatorInfo().getInputType());
 			assertEquals(initialData.getType(), reducer.getOperatorInfo().getOutputType());
 			
-			// DOP was not configured on the operator
-			assertTrue(reducer.getDegreeOfParallelism() == DOP || reducer.getDegreeOfParallelism() == -1);
+			// parallelism was not configured on the operator
+			assertTrue(reducer.getParallelism() == parallelism || reducer.getParallelism() == -1);
 			
 			// check keys
 			assertArrayEquals(new int[] {2}, reducer.getKeyColumns(0));
@@ -127,8 +127,8 @@ public class ReduceTranslationTests implements java.io.Serializable {
 	@Test
 	public void translateGroupedReduceWithkeyExtractor() {
 		try {
-			final int DOP = 8;
-			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(DOP);
+			final int parallelism = 8;
+			ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism);
 			
 			DataSet<Tuple3<Double, StringValue, LongValue>> initialData = getSourceDataSet(env);
 			
@@ -154,10 +154,10 @@ public class ReduceTranslationTests implements java.io.Serializable {
 			PlanUnwrappingReduceOperator<?, ?> reducer = (PlanUnwrappingReduceOperator<?, ?>) keyProjector.getInput();
 			MapOperatorBase<?, ?, ?> keyExtractor = (MapOperatorBase<?, ?, ?>) reducer.getInput();
 			
-			// check the DOPs
-			assertEquals(1, keyExtractor.getDegreeOfParallelism());
-			assertEquals(4, reducer.getDegreeOfParallelism());
-			assertEquals(4, keyProjector.getDegreeOfParallelism());
+			// check the parallelisms
+			assertEquals(1, keyExtractor.getParallelism());
+			assertEquals(4, reducer.getParallelism());
+			assertEquals(4, keyProjector.getParallelism());
 			
 			// check types
 			TypeInformation<?> keyValueInfo = new TupleTypeInfo<Tuple2<StringValue, Tuple3<Double,StringValue,LongValue>>>(

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
index 2101428..90421b7 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
@@ -288,9 +288,9 @@ public class Optimizer {
 	private final CostEstimator costEstimator;
 
 	/**
-	 * The default degree of parallelism for jobs compiled by this compiler.
+	 * The default parallelism for jobs compiled by this compiler.
 	 */
-	private int defaultDegreeOfParallelism;
+	private int defaultParallelism;
 
 
 	// ------------------------------------------------------------------------
@@ -348,14 +348,14 @@ public class Optimizer {
 		this.costEstimator = estimator;
 
 		// determine the default parallelism
-		this.defaultDegreeOfParallelism = GlobalConfiguration.getInteger(
-				ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY,
-				ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE);
+		this.defaultParallelism = GlobalConfiguration.getInteger(
+				ConfigConstants.DEFAULT_PARALLELISM_KEY,
+				ConfigConstants.DEFAULT_PARALLELISM);
 		
-		if (defaultDegreeOfParallelism < 1) {
-			LOG.warn("Config value " + defaultDegreeOfParallelism + " for option "
-					+ ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE + " is invalid. Ignoring and using a value of 1.");
-			this.defaultDegreeOfParallelism = 1;
+		if (defaultParallelism < 1) {
+			LOG.warn("Config value " + defaultParallelism + " for option "
+					+ ConfigConstants.DEFAULT_PARALLELISM + " is invalid. Ignoring and using a value of 1.");
+			this.defaultParallelism = 1;
 		}
 	}
 	
@@ -363,13 +363,13 @@ public class Optimizer {
 	//                             Getters / Setters
 	// ------------------------------------------------------------------------
 	
-	public int getDefaultDegreeOfParallelism() {
-		return defaultDegreeOfParallelism;
+	public int getDefaultParallelism() {
+		return defaultParallelism;
 	}
 	
-	public void setDefaultDegreeOfParallelism(int defaultDegreeOfParallelism) {
-		if (defaultDegreeOfParallelism > 0) {
-			this.defaultDegreeOfParallelism = defaultDegreeOfParallelism;
+	public void setDefaultParallelism(int defaultParallelism) {
+		if (defaultParallelism > 0) {
+			this.defaultParallelism = defaultParallelism;
 		} else {
 			throw new IllegalArgumentException("Default parallelism cannot be zero or negative.");
 		}
@@ -435,7 +435,7 @@ public class Optimizer {
 		final ExecutionMode defaultDataExchangeMode = program.getExecutionConfig().getExecutionMode();
 
 		final int defaultParallelism = program.getDefaultParallelism() > 0 ?
-			program.getDefaultParallelism() : this.defaultDegreeOfParallelism;
+			program.getDefaultParallelism() : this.defaultParallelism;
 
 		// log the default settings
 		LOG.debug("Using a default parallelism of {}",  defaultParallelism);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java
index 068799e..1600a50 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java
@@ -122,12 +122,12 @@ public class BinaryUnionNode extends TwoInputNode {
 		final ExecutionMode input1Mode = this.input1.getDataExchangeMode();
 		final ExecutionMode input2Mode = this.input2.getDataExchangeMode();
 
-		final int dop = getParallelism();
-		final int inDop1 = getFirstPredecessorNode().getParallelism();
-		final int inDop2 = getSecondPredecessorNode().getParallelism();
+		final int parallelism = getParallelism();
+		final int inParallelism1 = getFirstPredecessorNode().getParallelism();
+		final int inParallelism2 = getSecondPredecessorNode().getParallelism();
 
-		final boolean dopChange1 = dop != inDop1;
-		final boolean dopChange2 = dop != inDop2;
+		final boolean dopChange1 = parallelism != inParallelism1;
+		final boolean dopChange2 = parallelism != inParallelism2;
 
 		final boolean input1breakPipeline = this.input1.isBreakingPipeline();
 		final boolean input2breakPipeline = this.input2.isBreakingPipeline();
@@ -152,8 +152,8 @@ public class BinaryUnionNode extends TwoInputNode {
 						// free to choose the ship strategy
 						igps.parameterizeChannel(c1, dopChange1, input1Mode, input1breakPipeline);
 						
-						// if the DOP changed, make sure that we cancel out properties, unless the
-						// ship strategy preserves/establishes them even under changing DOPs
+						// if the parallelism changed, make sure that we cancel out properties, unless the
+						// ship strategy preserves/establishes them even under changing parallelisms
 						if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
 							c1.getGlobalProperties().reset();
 						}
@@ -179,8 +179,8 @@ public class BinaryUnionNode extends TwoInputNode {
 						// free to choose the ship strategy
 						igps.parameterizeChannel(c2, dopChange2, input2Mode, input2breakPipeline);
 						
-						// if the DOP changed, make sure that we cancel out properties, unless the
-						// ship strategy preserves/establishes them even under changing DOPs
+						// if the parallelism changed, make sure that we cancel out properties, unless the
+						// ship strategy preserves/establishes them even under changing parallelisms
 						if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) {
 							c2.getGlobalProperties().reset();
 						}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
index 55b8785..5dd868e 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
@@ -131,14 +131,14 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
 	 */
 	public void setNextPartialSolution(OptimizerNode nextPartialSolution, OptimizerNode terminationCriterion) {
 		
-		// check if the root of the step function has the same DOP as the iteration
+		// check if the root of the step function has the same parallelism as the iteration
 		// or if the step function has any operator at all
 		if (nextPartialSolution.getParallelism() != getParallelism() ||
 			nextPartialSolution == partialSolution || nextPartialSolution instanceof BinaryUnionNode)
 		{
 			// add a no-op to the root to express the re-partitioning
 			NoOpNode noop = new NoOpNode();
-			noop.setDegreeOfParallelism(getParallelism());
+			noop.setParallelism(getParallelism());
 
 			DagConnection noOpConn = new DagConnection(nextPartialSolution, noop, ExecutionMode.PIPELINED);
 			noop.setIncomingConnection(noOpConn);
@@ -323,7 +323,7 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
 					locPropsReq.parameterizeChannel(toNoOp);
 					
 					UnaryOperatorNode rebuildPropertiesNode = new UnaryOperatorNode("Rebuild Partial Solution Properties", FieldList.EMPTY_LIST);
-					rebuildPropertiesNode.setDegreeOfParallelism(candidate.getParallelism());
+					rebuildPropertiesNode.setParallelism(candidate.getParallelism());
 					
 					SingleInputPlanNode rebuildPropertiesPlanNode = new SingleInputPlanNode(rebuildPropertiesNode, "Rebuild Partial Solution Properties", toNoOp, DriverStrategy.UNARY_NO_OP);
 					rebuildPropertiesPlanNode.initProperties(toNoOp.getGlobalProperties(), toNoOp.getLocalProperties());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java
index dbe04f4..6ca1149 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java
@@ -204,11 +204,11 @@ public class DataSinkNode extends OptimizerNode {
 		List<? extends PlanNode> subPlans = getPredecessorNode().getAlternativePlans(estimator);
 		List<PlanNode> outputPlans = new ArrayList<PlanNode>();
 		
-		final int dop = getParallelism();
+		final int parallelism = getParallelism();
 		final int inDop = getPredecessorNode().getParallelism();
 
 		final ExecutionMode executionMode = this.input.getDataExchangeMode();
-		final boolean dopChange = dop != inDop;
+		final boolean dopChange = parallelism != inDop;
 		final boolean breakPipeline = this.input.isBreakingPipeline();
 
 		InterestingProperties ips = this.input.getInterestingProperties();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java
index e4b35b7..6010f6a 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java
@@ -75,7 +75,7 @@ public class DataSourceNode extends OptimizerNode {
 		}
 		
 		if (NonParallelInput.class.isAssignableFrom(pactContract.getUserCodeWrapper().getUserCodeClass())) {
-			setDegreeOfParallelism(1);
+			setParallelism(1);
 			this.sequentialInput = true;
 		} else {
 			this.sequentialInput = false;
@@ -115,10 +115,10 @@ public class DataSourceNode extends OptimizerNode {
 	}
 
 	@Override
-	public void setDegreeOfParallelism(int degreeOfParallelism) {
+	public void setParallelism(int parallelism) {
 		// if unsplittable, parallelism remains at 1
 		if (!this.sequentialInput) {
-			super.setDegreeOfParallelism(degreeOfParallelism);
+			super.setParallelism(parallelism);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java
index 564c0d3..d25fed9 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java
@@ -45,7 +45,7 @@ public class GroupCombineNode extends SingleInputNode {
 
 		if (this.keys == null) {
 			// case of a key-less reducer. force a parallelism of 1
-			setDegreeOfParallelism(1);
+			setParallelism(1);
 		}
 
 		this.possibleProperties = initPossibleProperties();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java
index 77acae5..227b75f 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java
@@ -53,7 +53,7 @@ public class GroupReduceNode extends SingleInputNode {
 		
 		if (this.keys == null) {
 			// case of a key-less reducer. force a parallelism of 1
-			setDegreeOfParallelism(1);
+			setParallelism(1);
 		}
 		
 		this.possibleProperties = initPossibleProperties(operator.getCustomPartitioner());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
index 0cad34e..6bf43ea 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
@@ -99,7 +99,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 	// --------------------------------- General Parameters ---------------------------------------
 	
 	private int parallelism = -1; // the number of parallel instances of this node
-	
+
 	private long minimalMemoryPerSubTask = -1;
 
 	protected int id = -1; 				// the id for this node.
@@ -390,9 +390,9 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
 	 * @param parallelism The parallelism to set.
 	 * @throws IllegalArgumentException If the parallelism is smaller than one and not -1.
 	 */
-	public void setDegreeOfParallelism(int parallelism) {
+	public void setParallelism(int parallelism) {
 		if (parallelism < 1 && parallelism != -1) {
-			throw new IllegalArgumentException("Degree of parallelism of " + parallelism + " is invalid.");
+			throw new IllegalArgumentException("Parallelism of " + parallelism + " is invalid.");
 		}
 		this.parallelism = parallelism;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java
index 1477038..52bfb6a 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java
@@ -43,7 +43,7 @@ public class ReduceNode extends SingleInputNode {
 		
 		if (this.keys == null) {
 			// case of a key-less reducer. force a parallelism of 1
-			setDegreeOfParallelism(1);
+			setParallelism(1);
 		}
 		
 		OperatorDescriptorSingle props = this.keys == null ?

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java
index cc12bb8..e9b31f4 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java
@@ -283,9 +283,10 @@ public abstract class SingleInputNode extends OptimizerNode {
 
 		final ExecutionMode executionMode = this.inConn.getDataExchangeMode();
 
-		final int dop = getParallelism();
-		final int inDop = getPredecessorNode().getParallelism();
-		final boolean dopChange = inDop != dop;
+		final int parallelism = getParallelism();
+		final int inParallelism = getPredecessorNode().getParallelism();
+
+		final boolean parallelismChange = inParallelism != parallelism;
 
 		final boolean breaksPipeline = this.inConn.isBreakingPipeline();
 
@@ -293,8 +294,8 @@ public abstract class SingleInputNode extends OptimizerNode {
 		for (PlanNode child : subPlans) {
 
 			if (child.getGlobalProperties().isFullyReplicated()) {
-				// fully replicated input is always locally forwarded if DOP is not changed
-				if (dopChange) {
+				// fully replicated input is always locally forwarded if the parallelism is not changed
+				if (parallelismChange) {
 					// can not continue with this child
 					childrenSkippedDueToReplicatedInput = true;
 					continue;
@@ -307,11 +308,11 @@ public 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, executionMode, breaksPipeline);
+					igps.parameterizeChannel(c, parallelismChange, executionMode, breaksPipeline);
 					
-					// if the DOP changed, make sure that we cancel out properties, unless the
-					// ship strategy preserves/establishes them even under changing DOPs
-					if (dopChange && !c.getShipStrategy().isNetworkStrategy()) {
+					// if the parallelism changed, make sure that we cancel out properties, unless the
+					// ship strategy preserves/establishes them even under changing parallelisms
+					if (parallelismChange && !c.getShipStrategy().isNetworkStrategy()) {
 						c.getGlobalProperties().reset();
 					}
 					
@@ -339,7 +340,7 @@ public abstract class SingleInputNode extends OptimizerNode {
 					c.setShipStrategy(shipStrategy, exMode);
 				}
 				
-				if (dopChange) {
+				if (parallelismChange) {
 					c.adjustGlobalPropertiesForFullParallelismChange();
 				}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java
index 40725ba..06606f0 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java
@@ -47,7 +47,7 @@ public class SinkJoiner extends TwoInputNode {
 		this.input1 = conn1;
 		this.input2 = conn2;
 		
-		setDegreeOfParallelism(1);
+		setParallelism(1);
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java
index 39da165..f3122ba 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java
@@ -352,12 +352,12 @@ public abstract class TwoInputNode extends OptimizerNode {
 		final ExecutionMode input1Mode = this.input1.getDataExchangeMode();
 		final ExecutionMode input2Mode = this.input2.getDataExchangeMode();
 
-		final int dop = getParallelism();
-		final int inDop1 = getFirstPredecessorNode().getParallelism();
-		final int inDop2 = getSecondPredecessorNode().getParallelism();
+		final int parallelism = getParallelism();
+		final int inParallelism1 = getFirstPredecessorNode().getParallelism();
+		final int inParallelism2 = getSecondPredecessorNode().getParallelism();
 
-		final boolean dopChange1 = dop != inDop1;
-		final boolean dopChange2 = dop != inDop2;
+		final boolean dopChange1 = parallelism != inParallelism1;
+		final boolean dopChange2 = parallelism != inParallelism2;
 
 		final boolean input1breaksPipeline = this.input1.isBreakingPipeline();
 		final boolean input2breaksPipeline = this.input2.isBreakingPipeline();
@@ -369,7 +369,7 @@ public abstract class TwoInputNode extends OptimizerNode {
 		for (PlanNode child1 : subPlans1) {
 
 			if (child1.getGlobalProperties().isFullyReplicated()) {
-				// fully replicated input is always locally forwarded if DOP is not changed
+				// fully replicated input is always locally forwarded if parallelism is not changed
 				if (dopChange1) {
 					// can not continue with this child
 					childrenSkippedDueToReplicatedInput = true;
@@ -382,7 +382,7 @@ public abstract class TwoInputNode extends OptimizerNode {
 			for (PlanNode child2 : subPlans2) {
 
 				if (child2.getGlobalProperties().isFullyReplicated()) {
-					// fully replicated input is always locally forwarded if DOP is not changed
+					// fully replicated input is always locally forwarded if parallelism is not changed
 					if (dopChange2) {
 						// can not continue with this child
 						childrenSkippedDueToReplicatedInput = true;
@@ -405,8 +405,8 @@ public abstract class TwoInputNode extends OptimizerNode {
 						// free to choose the ship strategy
 						igps1.parameterizeChannel(c1, dopChange1, input1Mode, input1breaksPipeline);
 						
-						// if the DOP changed, make sure that we cancel out properties, unless the
-						// ship strategy preserves/establishes them even under changing DOPs
+						// if the parallelism changed, make sure that we cancel out properties, unless the
+						// ship strategy preserves/establishes them even under changing parallelisms
 						if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) {
 							c1.getGlobalProperties().reset();
 						}
@@ -434,8 +434,8 @@ public abstract class TwoInputNode extends OptimizerNode {
 							// free to choose the ship strategy
 							igps2.parameterizeChannel(c2, dopChange2, input2Mode, input2breaksPipeline);
 							
-							// if the DOP changed, make sure that we cancel out properties, unless the
-							// ship strategy preserves/establishes them even under changing DOPs
+							// if the parallelism changed, make sure that we cancel out properties, unless the
+							// ship strategy preserves/establishes them even under changing parallelisms
 							if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) {
 								c2.getGlobalProperties().reset();
 							}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
index e85f289..99c868c 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
@@ -167,7 +167,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 		// if the next workset is equal to the workset, we need to inject a no-op node
 		if (nextWorkset == worksetNode || nextWorkset instanceof BinaryUnionNode) {
 			NoOpNode noop = new NoOpNode();
-			noop.setDegreeOfParallelism(getParallelism());
+			noop.setParallelism(getParallelism());
 
 			DagConnection noOpConn = new DagConnection(nextWorkset, noop, executionMode);
 			noop.setIncomingConnection(noOpConn);
@@ -179,7 +179,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 		// attach an extra node to the solution set delta for the cases where we need to repartition
 		UnaryOperatorNode solutionSetDeltaUpdateAux = new UnaryOperatorNode("Solution-Set Delta", getSolutionSetKeyFields(),
 				new SolutionSetDeltaOperator(getSolutionSetKeyFields()));
-		solutionSetDeltaUpdateAux.setDegreeOfParallelism(getParallelism());
+		solutionSetDeltaUpdateAux.setParallelism(getParallelism());
 
 		DagConnection conn = new DagConnection(solutionSetDelta, solutionSetDeltaUpdateAux, executionMode);
 		solutionSetDeltaUpdateAux.setIncomingConnection(conn);
@@ -371,7 +371,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 					UnaryOperatorNode rebuildWorksetPropertiesNode = new UnaryOperatorNode("Rebuild Workset Properties",
 																							FieldList.EMPTY_LIST);
 					
-					rebuildWorksetPropertiesNode.setDegreeOfParallelism(candidate.getParallelism());
+					rebuildWorksetPropertiesNode.setParallelism(candidate.getParallelism());
 					
 					SingleInputPlanNode rebuildWorksetPropertiesPlanNode = new SingleInputPlanNode(
 												rebuildWorksetPropertiesNode, "Rebuild Workset Properties",
@@ -563,7 +563,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 		SingleRootJoiner() {
 			super(new NoOpBinaryUdfOp<Nothing>(new NothingTypeInfo()));
 			
-			setDegreeOfParallelism(1);
+			setParallelism(1);
 		}
 		
 		public void setInputs(DagConnection input1, DagConnection input2) {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java
index 8c3f6bd..3646d74 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java
@@ -340,7 +340,7 @@ public final class RequestedGlobalProperties implements Cloneable {
 	 * the desired global properties.
 	 * 
 	 * @param channel The channel to parametrize.
-	 * @param globalDopChange Flag indicating whether the degree of parallelism changes
+	 * @param globalDopChange Flag indicating whether the parallelism changes
 	 *                        between sender and receiver.
 	 * @param exchangeMode The mode of data exchange (pipelined, always batch,
 	 *                     batch only on shuffle, ...)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java
index b3c083a..4990a5d 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java
@@ -53,9 +53,9 @@ public final class AllGroupWithPartialPreGroupProperties extends OperatorDescrip
 			Channel toCombiner = new Channel(in.getSource());
 			toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
 			
-			// create an input node for combine with same DOP as input node
+			// create an input node for combine with same parallelism as input node
 			GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode();
-			combinerNode.setDegreeOfParallelism(in.getSource().getParallelism());
+			combinerNode.setParallelism(in.getSource().getParallelism());
 
 			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode,
 					"Combine ("+node.getOperator().getName()+")", toCombiner, DriverStrategy.ALL_GROUP_REDUCE_COMBINE);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java
index a172a60..bd600e4 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java
@@ -52,9 +52,9 @@ public final class AllReduceProperties extends OperatorDescriptorSingle {
 			Channel toCombiner = new Channel(in.getSource());
 			toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
 			
-			// create an input node for combine with same DOP as input node
+			// create an input node for combine with same parallelism as input node
 			ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode();
-			combinerNode.setDegreeOfParallelism(in.getSource().getParallelism());
+			combinerNode.setParallelism(in.getSource().getParallelism());
 
 			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode,
 					"Combine ("+node.getOperator().getName()+")", toCombiner, DriverStrategy.ALL_REDUCE);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java
index b648386..64054a2 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java
@@ -71,7 +71,7 @@ public final class GroupCombineProperties extends OperatorDescriptorSingle {
 
 	@Override
 	public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
-		node.setDegreeOfParallelism(in.getSource().getParallelism());
+		node.setParallelism(in.getSource().getParallelism());
 		
 		// sorting key info
 		SingleInputPlanNode singleInputPlanNode = new SingleInputPlanNode(

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java
index c4f47d3..86863d2 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java
@@ -105,9 +105,9 @@ public final class GroupReduceWithCombineProperties extends OperatorDescriptorSi
 			Channel toCombiner = new Channel(in.getSource());
 			toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
 
-			// create an input node for combine with same DOP as input node
+			// create an input node for combine with same parallelism as input node
 			GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode();
-			combinerNode.setDegreeOfParallelism(in.getSource().getParallelism());
+			combinerNode.setParallelism(in.getSource().getParallelism());
 
 			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine("+node.getOperator()
 					.getName()+")", toCombiner, DriverStrategy.SORTED_GROUP_COMBINE);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java
index 2bde29b..e4e6a7f 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java
@@ -47,9 +47,9 @@ public final class PartialGroupProperties extends OperatorDescriptorSingle {
 
 	@Override
 	public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
-		// create in input node for combine with same DOP as input node
+		// create in input node for combine with the same parallelism as input node
 		GroupReduceNode combinerNode = new GroupReduceNode((GroupReduceOperatorBase<?, ?, ?>) node.getOperator());
-		combinerNode.setDegreeOfParallelism(in.getSource().getParallelism());
+		combinerNode.setParallelism(in.getSource().getParallelism());
 
 		SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode, "Combine("+node.getOperator().getName()+")", in,
 				DriverStrategy.SORTED_GROUP_COMBINE);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java
index 5bb51f3..81afe1e 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java
@@ -69,9 +69,9 @@ public final class ReduceProperties extends OperatorDescriptorSingle {
 			Channel toCombiner = new Channel(in.getSource());
 			toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
 			
-			// create an input node for combine with same DOP as input node
+			// create an input node for combine with same parallelism as input node
 			ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode();
-			combinerNode.setDegreeOfParallelism(in.getSource().getParallelism());
+			combinerNode.setParallelism(in.getSource().getParallelism());
 
 			SingleInputPlanNode combiner = new SingleInputPlanNode(combinerNode,
 								"Combine ("+node.getOperator().getName()+")", toCombiner,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
index 875d1c3..4f8b1be 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
@@ -473,7 +473,7 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
 	
 	public void adjustGlobalPropertiesForFullParallelismChange() {
 		if (this.shipStrategy == null || this.shipStrategy == ShipStrategyType.NONE) {
-			throw new IllegalStateException("Cannot adjust channel for degree of parallelism " +
+			throw new IllegalStateException("Cannot adjust channel for parallelism " +
 					"change before the ship strategy is set.");
 		}
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java
index 451484d..c93d8c2 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java
@@ -40,7 +40,7 @@ public class SinkJoinerPlanNode extends DualInputPlanNode {
 	public void setCosts(Costs nodeCosts) {
 		// the plan enumeration logic works as for regular two-input-operators, which is important
 		// because of the branch handling logic. it does pick redistributing network channels
-		// between the sink and the sink joiner, because sinks joiner has a different DOP than the sink.
+		// between the sink and the sink joiner, because sinks joiner has a different parallelism than the sink.
 		// we discard any cost and simply use the sum of the costs from the two children.
 		
 		Costs totalCosts = getInput1().getSource().getCumulativeCosts().clone();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
index 6f918c0..b04cdd8 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
@@ -275,7 +275,7 @@ public class PlanJSONDumpGenerator {
 		// output node contents
 		writer.print(",\n\t\t\"contents\": \"" + contents + "\"");
 
-		// degree of parallelism
+		// parallelism
 		writer.print(",\n\t\t\"parallelism\": \""
 			+ (n.getParallelism() >= 1 ? n.getParallelism() : "default") + "\"");
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
index 04bc527..dc21c13 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
@@ -264,15 +264,15 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 				// for the bulk iteration, we skip creating anything for now. we create the graph
 				// for the step function in the post visit.
 				
-				// check that the root of the step function has the same DOP as the iteration.
-				// because the tail must have the same DOP as the head, we can only merge the last
-				// operator with the tail, if they have the same DOP. not merging is currently not
+				// check that the root of the step function has the same parallelism as the iteration.
+				// because the tail must have the same parallelism as the head, we can only merge the last
+				// operator with the tail, if they have the same parallelism. not merging is currently not
 				// implemented
 				PlanNode root = iterationNode.getRootOfStepFunction();
 				if (root.getParallelism() != node.getParallelism())
 				{
 					throw new CompilerException("Error: The final operator of the step " +
-							"function has a different degree of parallelism than the iteration operator itself.");
+							"function has a different parallelism than the iteration operator itself.");
 				}
 				
 				IterationDescriptor descr = new IterationDescriptor(iterationNode, this.iterationIdEnumerator++);
@@ -289,12 +289,12 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 				if (nextWorkSet.getParallelism() != node.getParallelism())
 				{
 					throw new CompilerException("It is currently not supported that the final operator of the step " +
-							"function has a different degree of parallelism than the iteration operator itself.");
+							"function has a different parallelism than the iteration operator itself.");
 				}
 				if (solutionSetDelta.getParallelism() != node.getParallelism())
 				{
 					throw new CompilerException("It is currently not supported that the final operator of the step " +
-							"function has a different degree of parallelism than the iteration operator itself.");
+							"function has a different parallelism than the iteration operator itself.");
 				}
 				
 				IterationDescriptor descr = new IterationDescriptor(iterationNode, this.iterationIdEnumerator++);
@@ -362,7 +362,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 		
 		// check if a vertex was created, or if it was chained or skipped
 		if (vertex != null) {
-			// set degree of parallelism
+			// set parallelism
 			int pd = node.getParallelism();
 			vertex.setParallelism(pd);
 			
@@ -370,10 +370,10 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 			
 			// check whether this vertex is part of an iteration step function
 			if (this.currentIteration != null) {
-				// check that the task has the same DOP as the iteration as such
+				// check that the task has the same parallelism as the iteration as such
 				PlanNode iterationNode = (PlanNode) this.currentIteration;
 				if (iterationNode.getParallelism() < pd) {
-					throw new CompilerException("Error: All functions that are part of an iteration must have the same, or a lower, degree-of-parallelism than the iteration operator.");
+					throw new CompilerException("Error: All functions that are part of an iteration must have the same, or a lower, parallelism than the iteration operator.");
 				}
 
 				// store the id of the iterations the step functions participate in
@@ -725,7 +725,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 					return 1;
 				}
 				else {
-					throw new CompilerException("Error: A changing degree of parallelism is currently " +
+					throw new CompilerException("Error: A changing parallelism is currently " +
 							"not supported between tasks within an iteration.");
 				}
 			} else {
@@ -880,7 +880,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 		// the step function, if
 		// 1) There is one parent that the partial solution connects to via a forward pattern and no
 		//    local strategy
-		// 2) DOP and the number of subtasks per instance does not change
+		// 2) parallelism and the number of subtasks per instance does not change
 		// 3) That successor is not a union
 		// 4) That successor is not itself the last node of the step function
 		// 5) There is no local strategy on the edge for the initial partial solution, as
@@ -948,7 +948,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 		// the step function, if
 		// 1) There is one parent that the partial solution connects to via a forward pattern and no
 		//    local strategy
-		// 2) DOP and the number of subtasks per instance does not change
+		// 2) parallelism and the number of subtasks per instance does not change
 		// 3) That successor is not a union
 		// 4) That successor is not itself the last node of the step function
 		// 5) There is no local strategy on the edge for the initial workset, as

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
index 160ef95..37cffce 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java
@@ -83,20 +83,20 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 
 	private final List<DataSinkNode> sinks; // all data sink nodes in the optimizer plan
 
-	private final int defaultParallelism; // the default degree of parallelism
+	private final int defaultParallelism; // the default parallelism
 
 	private final GraphCreatingVisitor parent;	// reference to enclosing creator, in case of a recursive translation
 
 	private final ExecutionMode defaultDataExchangeMode;
 
-	private final boolean forceDOP;
+	private final boolean forceParallelism;
 
 
 	public GraphCreatingVisitor(int defaultParallelism, ExecutionMode defaultDataExchangeMode) {
 		this(null, false, defaultParallelism, defaultDataExchangeMode, null);
 	}
 
-	private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceDOP, int defaultParallelism,
+	private GraphCreatingVisitor(GraphCreatingVisitor parent, boolean forceParallelism, int defaultParallelism,
 									ExecutionMode dataExchangeMode, HashMap<Operator<?>, OptimizerNode> closure) {
 		if (closure == null){
 			con2node = new HashMap<Operator<?>, OptimizerNode>();
@@ -108,7 +108,7 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 		this.defaultParallelism = defaultParallelism;
 		this.parent = parent;
 		this.defaultDataExchangeMode = dataExchangeMode;
-		this.forceDOP = forceDOP;
+		this.forceParallelism = forceParallelism;
 	}
 
 	public List<DataSinkNode> getSinks() {
@@ -194,7 +194,7 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 
 			// catch this for the recursive translation of step functions
 			BulkPartialSolutionNode p = new BulkPartialSolutionNode(holder, containingIterationNode);
-			p.setDegreeOfParallelism(containingIterationNode.getParallelism());
+			p.setParallelism(containingIterationNode.getParallelism());
 			n = p;
 		}
 		else if (c instanceof DeltaIterationBase.WorksetPlaceHolder) {
@@ -209,7 +209,7 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 
 			// catch this for the recursive translation of step functions
 			WorksetNode p = new WorksetNode(holder, containingIterationNode);
-			p.setDegreeOfParallelism(containingIterationNode.getParallelism());
+			p.setParallelism(containingIterationNode.getParallelism());
 			n = p;
 		}
 		else if (c instanceof DeltaIterationBase.SolutionSetPlaceHolder) {
@@ -224,7 +224,7 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 
 			// catch this for the recursive translation of step functions
 			SolutionSetNode p = new SolutionSetNode(holder, containingIterationNode);
-			p.setDegreeOfParallelism(containingIterationNode.getParallelism());
+			p.setParallelism(containingIterationNode.getParallelism());
 			n = p;
 		}
 		else {
@@ -233,13 +233,13 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 
 		this.con2node.put(c, n);
 
-		// set the parallelism only if it has not been set before. some nodes have a fixed DOP, such as the
+		// set the parallelism only if it has not been set before. some nodes have a fixed parallelism, such as the
 		// key-less reducer (all-reduce)
 		if (n.getParallelism() < 1) {
-			// set the degree of parallelism
-			int par = c.getDegreeOfParallelism();
+			// set the parallelism
+			int par = c.getParallelism();
 			if (par > 0) {
-				if (this.forceDOP && par != this.defaultParallelism) {
+				if (this.forceParallelism && par != this.defaultParallelism) {
 					par = this.defaultParallelism;
 					Optimizer.LOG.warn("The parallelism of nested dataflows (such as step functions in iterations) is " +
 						"currently fixed to the parallelism of the surrounding operator (the iteration).");
@@ -247,7 +247,7 @@ public class GraphCreatingVisitor implements Visitor<Operator<?>> {
 			} else {
 				par = this.defaultParallelism;
 			}
-			n.setDegreeOfParallelism(par);
+			n.setParallelism(par);
 		}
 
 		return true;


[5/9] flink git commit: [FLINK-1679] use a consistent name for parallelism

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
index 916aa27..2df08a0 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
@@ -79,7 +79,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+			env.setParallelism(DEFAULT_PARALLELISM);
 
 			DataSet<Long> source = env.generateSequence(1, 10000);
 
@@ -120,7 +120,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	public void testBranchingWithMultipleDataSinks2() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+			env.setParallelism(DEFAULT_PARALLELISM);
 
 			DataSet<Long> source = env.generateSequence(1, 10000);
 
@@ -184,7 +184,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	public void testBranchingSourceMultipleTimes() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+			env.setParallelism(DEFAULT_PARALLELISM);
 
 			DataSet<Tuple2<Long, Long>> source = env.generateSequence(1, 10000000)
 				.map(new Duplicator<Long>());
@@ -267,7 +267,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	public void testBranchingWithMultipleDataSinks() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+			env.setParallelism(DEFAULT_PARALLELISM);
 
 			DataSet<Tuple2<Long, Long>> sourceA = env.generateSequence(1, 10000000)
 					.map(new Duplicator<Long>());
@@ -815,7 +815,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	public void testIterationWithStaticInput() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(100);
+			env.setParallelism(100);
 
 			DataSet<Long> source = env.generateSequence(1, 1000000);
 
@@ -842,7 +842,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	@Test
 	public void testBranchingBroadcastVariable() {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(100);
+		env.setParallelism(100);
 
 		DataSet<String> input1 = env.readTextFile(IN_FILE).name("source1");
 		DataSet<String> input2 = env.readTextFile(IN_FILE).name("source2");
@@ -914,7 +914,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	@Test
 	public void testMultipleIterations() {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(100);
+		env.setParallelism(100);
 		
 		DataSet<String> input = env.readTextFile(IN_FILE).name("source1");
 		
@@ -943,7 +943,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	@Test
 	public void testMultipleIterationsWithClosueBCVars() {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(100);
+		env.setParallelism(100);
 
 		DataSet<String> input = env.readTextFile(IN_FILE).name("source1");
 			
@@ -970,7 +970,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	public void testBranchesOnlyInBCVariables1() {
 		try{
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(100);
+			env.setParallelism(100);
 
 			DataSet<Long> input = env.generateSequence(1, 10);
 			DataSet<Long> bc_input = env.generateSequence(1, 10);
@@ -993,7 +993,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 	public void testBranchesOnlyInBCVariables2() {
 		try{
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(100);
+			env.setParallelism(100);
 
 			DataSet<Tuple2<Long, Long>> input = env.generateSequence(1, 10).map(new Duplicator<Long>()).name("proper input");
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java
index 3e7da6c..47efeb1 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java
@@ -203,7 +203,7 @@ public class CachedMatchStrategyCompilerTest extends CompilerTestBase {
 	private Plan getTestPlanRightStatic(String strategy) {
 		
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 		
 		DataSet<Tuple3<Long, Long, Long>> bigInput = env.readCsvFile("file://bigFile").types(Long.class, Long.class, Long.class).name("bigFile");
 		
@@ -231,7 +231,7 @@ public class CachedMatchStrategyCompilerTest extends CompilerTestBase {
 	private Plan getTestPlanLeftStatic(String strategy) {
 		
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 		
 		@SuppressWarnings("unchecked")
 		DataSet<Tuple3<Long, Long, Long>> bigInput = env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L),

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java
index 565d992..4eed236 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java
@@ -71,10 +71,10 @@ public abstract class CompilerTestBase implements java.io.Serializable {
 	public void setup() {
 		this.dataStats = new DataStatistics();
 		this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator());
-		this.withStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
+		this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
 		
 		this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator());
-		this.noStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
+		this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
 	}
 	
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/DOPChangeTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DOPChangeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DOPChangeTest.java
deleted file mode 100644
index b17e777..0000000
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DOPChangeTest.java
+++ /dev/null
@@ -1,347 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.optimizer;
-
-import org.junit.Assert;
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.java.record.operators.FileDataSink;
-import org.apache.flink.api.java.record.operators.FileDataSource;
-import org.apache.flink.api.java.record.operators.JoinOperator;
-import org.apache.flink.api.java.record.operators.MapOperator;
-import org.apache.flink.api.java.record.operators.ReduceOperator;
-import org.apache.flink.optimizer.plan.Channel;
-import org.apache.flink.optimizer.plan.DualInputPlanNode;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plan.PlanNode;
-import org.apache.flink.optimizer.plan.SingleInputPlanNode;
-import org.apache.flink.optimizer.plan.SinkPlanNode;
-import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
-import org.apache.flink.optimizer.util.DummyInputFormat;
-import org.apache.flink.optimizer.util.DummyMatchStub;
-import org.apache.flink.optimizer.util.DummyOutputFormat;
-import org.apache.flink.optimizer.util.IdentityMap;
-import org.apache.flink.optimizer.util.IdentityReduce;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.util.Visitor;
-import org.junit.Test;
-
-/**
- * Tests in this class:
- * <ul>
- *   <li>Tests that check the correct handling of the properties and strategies in the case where the degree of
- *       parallelism between tasks is increased or decreased.
- * </ul>
- */
-@SuppressWarnings({"serial", "deprecation"})
-public class DOPChangeTest extends CompilerTestBase {
-	
-	/**
-	 * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties).
-	 * 
-	 * Increases DOP between 1st reduce and 2nd map, so the hash partitioning from 1st reduce is not reusable.
-	 * Expected to re-establish partitioning between reduce and map, via hash, because random is a full network
-	 * transit as well.
-	 */
-	@Test
-	public void checkPropertyHandlingWithIncreasingGlobalParallelism1() {
-		final int degOfPar = DEFAULT_PARALLELISM;
-		
-		// construct the plan
-		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
-		source.setDegreeOfParallelism(degOfPar);
-		
-		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
-		map1.setDegreeOfParallelism(degOfPar);
-		map1.setInput(source);
-		
-		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
-		reduce1.setDegreeOfParallelism(degOfPar);
-		reduce1.setInput(map1);
-		
-		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
-		map2.setDegreeOfParallelism(degOfPar * 2);
-		map2.setInput(reduce1);
-		
-		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
-		reduce2.setDegreeOfParallelism(degOfPar * 2);
-		reduce2.setInput(map2);
-		
-		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
-		sink.setDegreeOfParallelism(degOfPar * 2);
-		sink.setInput(reduce2);
-		
-		Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism");
-		
-		// submit the plan to the compiler
-		OptimizedPlan oPlan = compileNoStats(plan);
-		
-		// check the optimized Plan
-		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
-		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
-		// mapper respectively reducer
-		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
-		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
-		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
-		
-		ShipStrategyType mapIn = map2Node.getInput().getShipStrategy();
-		ShipStrategyType redIn = red2Node.getInput().getShipStrategy();
-		
-		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.PARTITION_HASH, mapIn);
-		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, redIn);
-	}
-	
-	/**
-	 * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties).
-	 * 
-	 * Increases DOP between 2nd map and 2nd reduce, so the hash partitioning from 1st reduce is not reusable.
-	 * Expected to re-establish partitioning between map and reduce (hash).
-	 */
-	@Test
-	public void checkPropertyHandlingWithIncreasingGlobalParallelism2() {
-		final int degOfPar = DEFAULT_PARALLELISM;
-		
-		// construct the plan
-		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
-		source.setDegreeOfParallelism(degOfPar);
-		
-		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
-		map1.setDegreeOfParallelism(degOfPar);
-		map1.setInput(source);
-		
-		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
-		reduce1.setDegreeOfParallelism(degOfPar);
-		reduce1.setInput(map1);
-		
-		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
-		map2.setDegreeOfParallelism(degOfPar);
-		map2.setInput(reduce1);
-		
-		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
-		reduce2.setDegreeOfParallelism(degOfPar * 2);
-		reduce2.setInput(map2);
-		
-		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
-		sink.setDegreeOfParallelism(degOfPar * 2);
-		sink.setInput(reduce2);
-		
-		Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism");
-		
-		// submit the plan to the compiler
-		OptimizedPlan oPlan = compileNoStats(plan);
-		
-		// check the optimized Plan
-		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
-		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
-		// mapper respectively reducer
-		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
-		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
-		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
-		
-		ShipStrategyType mapIn = map2Node.getInput().getShipStrategy();
-		ShipStrategyType reduceIn = red2Node.getInput().getShipStrategy();
-		
-		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, mapIn);
-		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.PARTITION_HASH, reduceIn);
-	}
-	
-	/**
-	 * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties).
-	 * 
-	 * Increases DOP between 1st reduce and 2nd map, such that more tasks are on one instance.
-	 * Expected to re-establish partitioning between map and reduce via a local hash.
-	 */
-	@Test
-	public void checkPropertyHandlingWithIncreasingLocalParallelism() {
-		final int degOfPar = 2 * DEFAULT_PARALLELISM;
-		
-		// construct the plan
-		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
-		source.setDegreeOfParallelism(degOfPar);
-		
-		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
-		map1.setDegreeOfParallelism(degOfPar);
-		map1.setInput(source);
-		
-		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
-		reduce1.setDegreeOfParallelism(degOfPar);
-		reduce1.setInput(map1);
-		
-		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
-		map2.setDegreeOfParallelism(degOfPar * 2);
-		map2.setInput(reduce1);
-		
-		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
-		reduce2.setDegreeOfParallelism(degOfPar * 2);
-		reduce2.setInput(map2);
-		
-		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
-		sink.setDegreeOfParallelism(degOfPar * 2);
-		sink.setInput(reduce2);
-		
-		Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism");
-		
-		// submit the plan to the compiler
-		OptimizedPlan oPlan = compileNoStats(plan);
-		
-		// check the optimized Plan
-		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
-		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
-		// mapper respectively reducer
-		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
-		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
-		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
-		
-		ShipStrategyType mapIn = map2Node.getInput().getShipStrategy();
-		ShipStrategyType reduceIn = red2Node.getInput().getShipStrategy();
-		
-		Assert.assertTrue("Invalid ship strategy for an operator.", 
-				(ShipStrategyType.PARTITION_RANDOM ==  mapIn && ShipStrategyType.PARTITION_HASH == reduceIn) || 
-				(ShipStrategyType.PARTITION_HASH == mapIn && ShipStrategyType.FORWARD == reduceIn));
-	}
-	
-	
-	
-	@Test
-	public void checkPropertyHandlingWithDecreasingDegreeOfParallelism() {
-		final int degOfPar = DEFAULT_PARALLELISM;
-		
-		// construct the plan
-		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
-		source.setDegreeOfParallelism(degOfPar * 2);
-		
-		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
-		map1.setDegreeOfParallelism(degOfPar * 2);
-		map1.setInput(source);
-		
-		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
-		reduce1.setDegreeOfParallelism(degOfPar * 2);
-		reduce1.setInput(map1);
-		
-		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
-		map2.setDegreeOfParallelism(degOfPar);
-		map2.setInput(reduce1);
-		
-		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
-		reduce2.setDegreeOfParallelism(degOfPar);
-		reduce2.setInput(map2);
-		
-		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
-		sink.setDegreeOfParallelism(degOfPar);
-		sink.setInput(reduce2);
-		
-		Plan plan = new Plan(sink, "Test Increasing Degree Of Parallelism");
-		
-		// submit the plan to the compiler
-		OptimizedPlan oPlan = compileNoStats(plan);
-
-		// check the optimized Plan
-		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
-		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
-		// mapper respectively reducer
-		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
-		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
-		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
-
-		Assert.assertTrue("The no sorting local strategy.",
-				LocalStrategy.SORT == red2Node.getInput().getLocalStrategy() ||
-						LocalStrategy.SORT == map2Node.getInput().getLocalStrategy());
-
-		Assert.assertTrue("The no partitioning ship strategy.",
-				ShipStrategyType.PARTITION_HASH == red2Node.getInput().getShipStrategy() ||
-						ShipStrategyType.PARTITION_HASH == map2Node.getInput().getShipStrategy());
-	}
-
-	/**
-	 * Checks that re-partitioning happens when the inputs of a two-input contract have different DOPs.
-	 * 
-	 * Test Plan:
-	 * <pre>
-	 * 
-	 * (source) -> reduce -\
-	 *                      Match -> (sink)
-	 * (source) -> reduce -/
-	 * 
-	 * </pre>
-	 * 
-	 */
-	@Test
-	public void checkPropertyHandlingWithTwoInputs() {
-		// construct the plan
-
-		FileDataSource sourceA = new FileDataSource(new DummyInputFormat(), IN_FILE);
-		FileDataSource sourceB = new FileDataSource(new DummyInputFormat(), IN_FILE);
-		
-		ReduceOperator redA = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0)
-			.input(sourceA)
-			.build();
-		ReduceOperator redB = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0)
-			.input(sourceB)
-			.build();
-		
-		JoinOperator mat = JoinOperator.builder(new DummyMatchStub(), IntValue.class, 0, 0)
-			.input1(redA)
-			.input2(redB)
-			.build();
-		
-		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, mat);
-		
-		sourceA.setDegreeOfParallelism(5);
-		sourceB.setDegreeOfParallelism(7);
-		redA.setDegreeOfParallelism(5);
-		redB.setDegreeOfParallelism(7);
-		
-		mat.setDegreeOfParallelism(5);
-		
-		sink.setDegreeOfParallelism(5);
-		
-		
-		// return the PACT plan
-		Plan plan = new Plan(sink, "Partition on DoP Change");
-		
-		OptimizedPlan oPlan = compileNoStats(plan);
-		
-		JobGraphGenerator jobGen = new JobGraphGenerator();
-		
-		//Compile plan to verify that no error is thrown
-		jobGen.compileJobGraph(oPlan);
-		
-		oPlan.accept(new Visitor<PlanNode>() {
-			
-			@Override
-			public boolean preVisit(PlanNode visitable) {
-				if (visitable instanceof DualInputPlanNode) {
-					DualInputPlanNode node = (DualInputPlanNode) visitable;
-					Channel c1 = node.getInput1();
-					Channel c2 = node.getInput2();
-					
-					Assert.assertEquals("Incompatible shipping strategy chosen for match", ShipStrategyType.FORWARD, c1.getShipStrategy());
-					Assert.assertEquals("Incompatible shipping strategy chosen for match", ShipStrategyType.PARTITION_HASH, c2.getShipStrategy());
-					return false;
-				}
-				return true;
-			}
-			
-			@Override
-			public void postVisit(PlanNode visitable) {
-				// DO NOTHING
-			}
-		});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java
index 34aa9f8..3b7eae7 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java
@@ -42,7 +42,7 @@ public class DistinctCompilationTest extends CompilerTestBase implements java.io
 	public void testDistinctPlain() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 					.name("source").setParallelism(6);
@@ -77,7 +77,7 @@ public class DistinctCompilationTest extends CompilerTestBase implements java.io
 			assertEquals(new FieldList(0, 1), combineNode.getKeys(0));
 			assertEquals(new FieldList(0, 1), reduceNode.getInput().getLocalStrategyKeys());
 
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, combineNode.getParallelism());
 			assertEquals(8, reduceNode.getParallelism());
@@ -94,7 +94,7 @@ public class DistinctCompilationTest extends CompilerTestBase implements java.io
 	public void testDistinctWithSelectorFunctionKey() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 					.name("source").setParallelism(6);
@@ -135,7 +135,7 @@ public class DistinctCompilationTest extends CompilerTestBase implements java.io
 			assertEquals(new FieldList(0), combineNode.getKeys(0));
 			assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys());
 
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, keyExtractor.getParallelism());
 			assertEquals(6, combineNode.getParallelism());
@@ -155,7 +155,7 @@ public class DistinctCompilationTest extends CompilerTestBase implements java.io
 	public void testDistinctWithFieldPositionKeyCombinable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 					.name("source").setParallelism(6);
@@ -191,7 +191,7 @@ public class DistinctCompilationTest extends CompilerTestBase implements java.io
 			assertEquals(new FieldList(1), combineNode.getKeys(0));
 			assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys());
 
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, combineNode.getParallelism());
 			assertEquals(8, reduceNode.getParallelism());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java
index ac4f820..810ec0e 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java
@@ -91,7 +91,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 	public void testTwoIterationsWithMapperInbetween() throws Exception {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<Long, Long>> verticesWithInitialId = env.fromElements(new Tuple2<Long, Long>(1L, 2L));
 			
@@ -129,7 +129,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 	public void testTwoIterationsDirectlyChained() throws Exception {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<Long, Long>> verticesWithInitialId = env.fromElements(new Tuple2<Long, Long>(1L, 2L));
 			
@@ -165,7 +165,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 	public void testTwoWorksetIterationsDirectlyChained() throws Exception {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<Long, Long>> verticesWithInitialId = env.fromElements(new Tuple2<Long, Long>(1L, 2L));
 			
@@ -201,7 +201,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 	public void testIterationPushingWorkOut() throws Exception {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<Long, Long>> input1 = env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue());
 			
@@ -235,7 +235,7 @@ public class IterationsCompilerTest extends CompilerTestBase {
 	public void testWorksetIterationPipelineBreakerPlacement() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			// the workset (input two of the delta iteration) is the same as what is consumed be the successive join
 			DataSet<Tuple2<Long, Long>> initialWorkset = env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java
new file mode 100644
index 0000000..a54136a
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java
@@ -0,0 +1,347 @@
+/*
+ * 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.optimizer;
+
+import org.apache.flink.optimizer.plan.Channel;
+import org.apache.flink.optimizer.plan.DualInputPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.PlanNode;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.DummyInputFormat;
+import org.apache.flink.optimizer.util.DummyMatchStub;
+import org.apache.flink.optimizer.util.DummyOutputFormat;
+import org.apache.flink.optimizer.util.IdentityMap;
+import org.apache.flink.optimizer.util.IdentityReduce;
+import org.junit.Assert;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.java.record.operators.FileDataSink;
+import org.apache.flink.api.java.record.operators.FileDataSource;
+import org.apache.flink.api.java.record.operators.JoinOperator;
+import org.apache.flink.api.java.record.operators.MapOperator;
+import org.apache.flink.api.java.record.operators.ReduceOperator;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.runtime.operators.util.LocalStrategy;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.util.Visitor;
+import org.junit.Test;
+
+/**
+ * Tests in this class:
+ * <ul>
+ *   <li>Tests that check the correct handling of the properties and strategies in the case where the
+ *       parallelism between tasks is increased or decreased.
+ * </ul>
+ */
+@SuppressWarnings({"serial", "deprecation"})
+public class ParallelismChangeTest extends CompilerTestBase {
+	
+	/**
+	 * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties).
+	 * 
+	 * Increases parallelism between 1st reduce and 2nd map, so the hash partitioning from 1st reduce is not reusable.
+	 * Expected to re-establish partitioning between reduce and map, via hash, because random is a full network
+	 * transit as well.
+	 */
+	@Test
+	public void checkPropertyHandlingWithIncreasingGlobalParallelism1() {
+		final int degOfPar = DEFAULT_PARALLELISM;
+		
+		// construct the plan
+		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
+		source.setParallelism(degOfPar);
+		
+		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
+		map1.setParallelism(degOfPar);
+		map1.setInput(source);
+		
+		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
+		reduce1.setParallelism(degOfPar);
+		reduce1.setInput(map1);
+		
+		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
+		map2.setParallelism(degOfPar * 2);
+		map2.setInput(reduce1);
+		
+		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
+		reduce2.setParallelism(degOfPar * 2);
+		reduce2.setInput(map2);
+		
+		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
+		sink.setParallelism(degOfPar * 2);
+		sink.setInput(reduce2);
+		
+		Plan plan = new Plan(sink, "Test Increasing parallelism");
+		
+		// submit the plan to the compiler
+		OptimizedPlan oPlan = compileNoStats(plan);
+		
+		// check the optimized Plan
+		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
+		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
+		// mapper respectively reducer
+		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
+		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
+		
+		ShipStrategyType mapIn = map2Node.getInput().getShipStrategy();
+		ShipStrategyType redIn = red2Node.getInput().getShipStrategy();
+		
+		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.PARTITION_HASH, mapIn);
+		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, redIn);
+	}
+	
+	/**
+	 * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties).
+	 * 
+	 * Increases parallelism between 2nd map and 2nd reduce, so the hash partitioning from 1st reduce is not reusable.
+	 * Expected to re-establish partitioning between map and reduce (hash).
+	 */
+	@Test
+	public void checkPropertyHandlingWithIncreasingGlobalParallelism2() {
+		final int degOfPar = DEFAULT_PARALLELISM;
+		
+		// construct the plan
+		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
+		source.setParallelism(degOfPar);
+		
+		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
+		map1.setParallelism(degOfPar);
+		map1.setInput(source);
+		
+		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
+		reduce1.setParallelism(degOfPar);
+		reduce1.setInput(map1);
+		
+		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
+		map2.setParallelism(degOfPar);
+		map2.setInput(reduce1);
+		
+		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
+		reduce2.setParallelism(degOfPar * 2);
+		reduce2.setInput(map2);
+		
+		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
+		sink.setParallelism(degOfPar * 2);
+		sink.setInput(reduce2);
+		
+		Plan plan = new Plan(sink, "Test Increasing parallelism");
+		
+		// submit the plan to the compiler
+		OptimizedPlan oPlan = compileNoStats(plan);
+		
+		// check the optimized Plan
+		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
+		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
+		// mapper respectively reducer
+		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
+		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
+		
+		ShipStrategyType mapIn = map2Node.getInput().getShipStrategy();
+		ShipStrategyType reduceIn = red2Node.getInput().getShipStrategy();
+		
+		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, mapIn);
+		Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.PARTITION_HASH, reduceIn);
+	}
+	
+	/**
+	 * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all properties).
+	 * 
+	 * Increases parallelism between 1st reduce and 2nd map, such that more tasks are on one instance.
+	 * Expected to re-establish partitioning between map and reduce via a local hash.
+	 */
+	@Test
+	public void checkPropertyHandlingWithIncreasingLocalParallelism() {
+		final int degOfPar = 2 * DEFAULT_PARALLELISM;
+		
+		// construct the plan
+		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
+		source.setParallelism(degOfPar);
+		
+		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
+		map1.setParallelism(degOfPar);
+		map1.setInput(source);
+		
+		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
+		reduce1.setParallelism(degOfPar);
+		reduce1.setInput(map1);
+		
+		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
+		map2.setParallelism(degOfPar * 2);
+		map2.setInput(reduce1);
+		
+		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
+		reduce2.setParallelism(degOfPar * 2);
+		reduce2.setInput(map2);
+		
+		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
+		sink.setParallelism(degOfPar * 2);
+		sink.setInput(reduce2);
+		
+		Plan plan = new Plan(sink, "Test Increasing parallelism");
+		
+		// submit the plan to the compiler
+		OptimizedPlan oPlan = compileNoStats(plan);
+		
+		// check the optimized Plan
+		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
+		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
+		// mapper respectively reducer
+		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
+		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
+		
+		ShipStrategyType mapIn = map2Node.getInput().getShipStrategy();
+		ShipStrategyType reduceIn = red2Node.getInput().getShipStrategy();
+		
+		Assert.assertTrue("Invalid ship strategy for an operator.", 
+				(ShipStrategyType.PARTITION_RANDOM ==  mapIn && ShipStrategyType.PARTITION_HASH == reduceIn) || 
+				(ShipStrategyType.PARTITION_HASH == mapIn && ShipStrategyType.FORWARD == reduceIn));
+	}
+	
+	
+	
+	@Test
+	public void checkPropertyHandlingWithDecreasingParallelism() {
+		final int degOfPar = DEFAULT_PARALLELISM;
+		
+		// construct the plan
+		FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
+		source.setParallelism(degOfPar * 2);
+		
+		MapOperator map1 = MapOperator.builder(new IdentityMap()).name("Map1").build();
+		map1.setParallelism(degOfPar * 2);
+		map1.setInput(source);
+		
+		ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 1").build();
+		reduce1.setParallelism(degOfPar * 2);
+		reduce1.setInput(map1);
+		
+		MapOperator map2 = MapOperator.builder(new IdentityMap()).name("Map2").build();
+		map2.setParallelism(degOfPar);
+		map2.setInput(reduce1);
+		
+		ReduceOperator reduce2 = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).name("Reduce 2").build();
+		reduce2.setParallelism(degOfPar);
+		reduce2.setInput(map2);
+		
+		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
+		sink.setParallelism(degOfPar);
+		sink.setInput(reduce2);
+		
+		Plan plan = new Plan(sink, "Test Increasing parallelism");
+		
+		// submit the plan to the compiler
+		OptimizedPlan oPlan = compileNoStats(plan);
+
+		// check the optimized Plan
+		// when reducer 1 distributes its data across the instances of map2, it needs to employ a local hash method,
+		// because map2 has twice as many instances and key/value pairs with the same key need to be processed by the same
+		// mapper respectively reducer
+		SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+		SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor();
+		SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor();
+
+		Assert.assertTrue("The no sorting local strategy.",
+				LocalStrategy.SORT == red2Node.getInput().getLocalStrategy() ||
+						LocalStrategy.SORT == map2Node.getInput().getLocalStrategy());
+
+		Assert.assertTrue("The no partitioning ship strategy.",
+				ShipStrategyType.PARTITION_HASH == red2Node.getInput().getShipStrategy() ||
+						ShipStrategyType.PARTITION_HASH == map2Node.getInput().getShipStrategy());
+	}
+
+	/**
+	 * Checks that re-partitioning happens when the inputs of a two-input contract have different parallelisms.
+	 * 
+	 * Test Plan:
+	 * <pre>
+	 * 
+	 * (source) -> reduce -\
+	 *                      Match -> (sink)
+	 * (source) -> reduce -/
+	 * 
+	 * </pre>
+	 * 
+	 */
+	@Test
+	public void checkPropertyHandlingWithTwoInputs() {
+		// construct the plan
+
+		FileDataSource sourceA = new FileDataSource(new DummyInputFormat(), IN_FILE);
+		FileDataSource sourceB = new FileDataSource(new DummyInputFormat(), IN_FILE);
+		
+		ReduceOperator redA = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0)
+			.input(sourceA)
+			.build();
+		ReduceOperator redB = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0)
+			.input(sourceB)
+			.build();
+		
+		JoinOperator mat = JoinOperator.builder(new DummyMatchStub(), IntValue.class, 0, 0)
+			.input1(redA)
+			.input2(redB)
+			.build();
+		
+		FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, mat);
+		
+		sourceA.setParallelism(5);
+		sourceB.setParallelism(7);
+		redA.setParallelism(5);
+		redB.setParallelism(7);
+		
+		mat.setParallelism(5);
+		
+		sink.setParallelism(5);
+		
+		
+		// return the PACT plan
+		Plan plan = new Plan(sink, "Partition on DoP Change");
+		
+		OptimizedPlan oPlan = compileNoStats(plan);
+		
+		JobGraphGenerator jobGen = new JobGraphGenerator();
+		
+		//Compile plan to verify that no error is thrown
+		jobGen.compileJobGraph(oPlan);
+		
+		oPlan.accept(new Visitor<PlanNode>() {
+			
+			@Override
+			public boolean preVisit(PlanNode visitable) {
+				if (visitable instanceof DualInputPlanNode) {
+					DualInputPlanNode node = (DualInputPlanNode) visitable;
+					Channel c1 = node.getInput1();
+					Channel c2 = node.getInput2();
+					
+					Assert.assertEquals("Incompatible shipping strategy chosen for match", ShipStrategyType.FORWARD, c1.getShipStrategy());
+					Assert.assertEquals("Incompatible shipping strategy chosen for match", ShipStrategyType.PARTITION_HASH, c2.getShipStrategy());
+					return false;
+				}
+				return true;
+			}
+			
+			@Override
+			public void postVisit(PlanNode visitable) {
+				// DO NOTHING
+			}
+		});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java
index 86f01b0..31f71d1 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java
@@ -41,7 +41,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 	public void testPipelineBreakerWithBroadcastVariable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(64);
+			env.setParallelism(64);
 			
 			DataSet<Long> source = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
 			
@@ -69,7 +69,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 	public void testPipelineBreakerBroadcastedAllReduce() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(64);
+			env.setParallelism(64);
 			
 			DataSet<Long> sourceWithMapper = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
 			
@@ -103,7 +103,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 	public void testPipelineBreakerBroadcastedPartialSolution() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(64);
+			env.setParallelism(64);
 			
 			
 			DataSet<Long> initialSource = env.generateSequence(1, 10);
@@ -144,7 +144,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 		try {
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
 				
@@ -166,7 +166,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 			
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
 				
@@ -189,7 +189,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 			
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
 				
@@ -212,7 +212,7 @@ public class PipelineBreakerTest extends CompilerTestBase {
 			
 			{
 				ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				env.setDegreeOfParallelism(64);
+				env.setParallelism(64);
 				
 				DataSet<Long> initialSource = env.generateSequence(1, 10);
 				

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java
index 7be2b16..3cf081f 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java
@@ -55,7 +55,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -88,7 +88,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -121,7 +121,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource3() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -153,7 +153,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource4() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -185,7 +185,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource5() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -217,7 +217,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource6() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -249,7 +249,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedSource7() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -283,7 +283,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -317,7 +317,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -352,7 +352,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource3() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -386,7 +386,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource4() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -420,7 +420,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource5() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -454,7 +454,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource6() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -488,7 +488,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource7() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -521,7 +521,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedGroupedSource8() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -555,7 +555,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedOrderedSource1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -589,7 +589,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedOrderedSource2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -624,7 +624,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedOrderedSource3() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -658,7 +658,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedOrderedSource4() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -692,7 +692,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedOrderedSource5() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -726,7 +726,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedOrderedSource6() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -759,7 +759,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkSinglePartitionedOrderedSource7() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple3<Long, SomePojo, String>> data = env.fromCollection(tuple3PojoData, tuple3PojoType);
 
@@ -793,7 +793,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkCoPartitionedSources1() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data1 =
 				env.readCsvFile("/some/path").types(Long.class, String.class);
@@ -841,7 +841,7 @@ public class PropertyDataSourceTest extends CompilerTestBase {
 	public void checkCoPartitionedSources2() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		DataSource<Tuple2<Long, String>> data1 =
 				env.readCsvFile("/some/path").types(Long.class, String.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
index da44b59..fd451f7 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
@@ -53,7 +53,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkJoinWithReplicatedSourceInput() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -89,7 +89,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkJoinWithReplicatedSourceInputBehindMap() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -126,7 +126,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkJoinWithReplicatedSourceInputBehindFilter() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -163,7 +163,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkJoinWithReplicatedSourceInputBehindFlatMap() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -200,7 +200,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkJoinWithReplicatedSourceInputBehindMapPartition() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -237,7 +237,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkJoinWithReplicatedSourceInputBehindMultiMaps() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -277,7 +277,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkCrossWithReplicatedSourceInput() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -313,7 +313,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	public void checkCrossWithReplicatedSourceInputBehindMap() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -345,13 +345,13 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	}
 
 	/**
-	 * Tests compiler fail for join program with replicated data source and changing DOP.
+	 * Tests compiler fail for join program with replicated data source and changing parallelism.
 	 */
 	@Test(expected = CompilerException.class)
-	public void checkJoinWithReplicatedSourceInputChangingDOP() {
+	public void checkJoinWithReplicatedSourceInputChangingparallelism() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -370,13 +370,13 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	}
 
 	/**
-	 * Tests compiler fail for join program with replicated data source behind map and changing DOP.
+	 * Tests compiler fail for join program with replicated data source behind map and changing parallelism.
 	 */
 	@Test(expected = CompilerException.class)
-	public void checkJoinWithReplicatedSourceInputBehindMapChangingDOP() {
+	public void checkJoinWithReplicatedSourceInputBehindMapChangingparallelism() {
 
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -402,7 +402,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	@Test(expected = CompilerException.class)
 	public void checkJoinWithReplicatedSourceInputBehindReduce() {
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
@@ -427,7 +427,7 @@ public class ReplicatingDataSourceTest extends CompilerTestBase {
 	@Test(expected = CompilerException.class)
 	public void checkJoinWithReplicatedSourceInputBehindRebalance() {
 		ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 
 		ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
 				new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java
index d397ea2..f865a9f 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java
@@ -45,7 +45,7 @@ public class CustomPartitioningTest extends CompilerTestBase {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet<Tuple2<Integer, Integer>> data = env.fromElements(new Tuple2<Integer,Integer>(0, 0))
 					.rebalance();
@@ -88,7 +88,7 @@ public class CustomPartitioningTest extends CompilerTestBase {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet<Tuple2<Integer, Integer>> data = env.fromElements(new Tuple2<Integer,Integer>(0, 0))
 					.rebalance();
@@ -115,7 +115,7 @@ public class CustomPartitioningTest extends CompilerTestBase {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet<Pojo> data = env.fromElements(new Pojo())
 					.rebalance();
@@ -158,7 +158,7 @@ public class CustomPartitioningTest extends CompilerTestBase {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet<Pojo> data = env.fromElements(new Pojo())
 					.rebalance();
@@ -185,7 +185,7 @@ public class CustomPartitioningTest extends CompilerTestBase {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet<Pojo> data = env.fromElements(new Pojo())
 					.rebalance();
@@ -237,7 +237,7 @@ public class CustomPartitioningTest extends CompilerTestBase {
 			final int parallelism = 4;
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(parallelism);
+			env.setParallelism(parallelism);
 			
 			DataSet<Pojo> data = env.fromElements(new Pojo())
 					.rebalance();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java
index de02836..2f9b32f 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java
@@ -39,7 +39,7 @@ public class DistinctAndGroupingOptimizerTest extends CompilerTestBase {
 	public void testDistinctPreservesPartitioningOfDistinctFields() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(4);
+			env.setParallelism(4);
 			
 			@SuppressWarnings("unchecked")
 			DataSet<Tuple2<Long, Long>> data = env.fromElements(new Tuple2<Long, Long>(0L, 0L), new Tuple2<Long, Long>(1L, 1L))
@@ -75,7 +75,7 @@ public class DistinctAndGroupingOptimizerTest extends CompilerTestBase {
 	public void testDistinctDestroysPartitioningOfNonDistinctFields() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(4);
+			env.setParallelism(4);
 			
 			@SuppressWarnings("unchecked")
 			DataSet<Tuple2<Long, Long>> data = env.fromElements(new Tuple2<Long, Long>(0L, 0L), new Tuple2<Long, Long>(1L, 1L))

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java
index a683968..c0e2fa7 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java
@@ -44,7 +44,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 	public void testAllGroupReduceNoCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Double> data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source");
 			
@@ -59,7 +59,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 			OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op);
 			
 			
-			// the all-reduce has no combiner, when the DOP of the input is one
+			// the all-reduce has no combiner, when the parallelism of the input is one
 			
 			SourcePlanNode sourceNode = resolver.getNode("source");
 			SingleInputPlanNode reduceNode = resolver.getNode("reducer");
@@ -72,7 +72,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 			// check that reduce has the right strategy
 			assertEquals(DriverStrategy.ALL_GROUP_REDUCE, reduceNode.getDriverStrategy());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(1, sourceNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
 			assertEquals(1, sinkNode.getParallelism());
@@ -88,7 +88,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 	public void testAllReduceWithCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Long> data = env.generateSequence(1, 8000000).name("source");
 			
@@ -120,7 +120,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 			assertEquals(DriverStrategy.ALL_GROUP_REDUCE, reduceNode.getDriverStrategy());
 			assertEquals(DriverStrategy.ALL_GROUP_REDUCE_COMBINE, combineNode.getDriverStrategy());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(8, sourceNode.getParallelism());
 			assertEquals(8, combineNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
@@ -138,7 +138,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 	public void testGroupedReduceWithFieldPositionKeyNonCombinable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -171,7 +171,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 			assertEquals(new FieldList(1), reduceNode.getKeys(0));
 			assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(8, reduceNode.getParallelism());
 			assertEquals(8, sinkNode.getParallelism());
@@ -187,7 +187,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 	public void testGroupedReduceWithFieldPositionKeyCombinable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -228,7 +228,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 			assertEquals(new FieldList(1), combineNode.getKeys(1));
 			assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, combineNode.getParallelism());
 			assertEquals(8, reduceNode.getParallelism());
@@ -245,7 +245,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 	public void testGroupedReduceWithSelectorFunctionKeyNoncombinable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -284,7 +284,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 			assertEquals(new FieldList(0), reduceNode.getKeys(0));
 			assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, keyExtractor.getParallelism());
 			
@@ -303,7 +303,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 	public void testGroupedReduceWithSelectorFunctionKeyCombinable() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -350,7 +350,7 @@ public class GroupReduceCompilationTest extends CompilerTestBase implements java
 			assertEquals(new FieldList(0), combineNode.getKeys(1));
 			assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, keyExtractor.getParallelism());
 			assertEquals(6, combineNode.getParallelism());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java
index 37a8e81..57d2d54 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java
@@ -45,7 +45,7 @@ public class IterationCompilerTest extends CompilerTestBase {
 	public void testIdentityIteration() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(43);
+			env.setParallelism(43);
 			
 			IterativeDataSet<Long> iteration = env.generateSequence(-4, 1000).iterate(100);
 			iteration.closeWith(iteration).print();
@@ -65,7 +65,7 @@ public class IterationCompilerTest extends CompilerTestBase {
 	public void testEmptyWorksetIteration() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(43);
+			env.setParallelism(43);
 			
 			DataSet<Tuple2<Long, Long>> input = env.generateSequence(1, 20)
 					.map(new MapFunction<Long, Tuple2<Long, Long>>() {
@@ -93,7 +93,7 @@ public class IterationCompilerTest extends CompilerTestBase {
 	public void testIterationWithUnionRoot() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(43);
+			env.setParallelism(43);
 			
 			IterativeDataSet<Long> iteration = env.generateSequence(-4, 1000).iterate(100);
 			
@@ -132,7 +132,7 @@ public class IterationCompilerTest extends CompilerTestBase {
 	public void testWorksetIterationWithUnionRoot() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(43);
+			env.setParallelism(43);
 			
 			DataSet<Tuple2<Long, Long>> input = env.generateSequence(1, 20)
 					.map(new MapFunction<Long, Tuple2<Long, Long>>() {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java
index 0724a9f..e1b18f9 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java
@@ -42,7 +42,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 	public void testAllReduceNoCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Double> data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source");
 			
@@ -61,7 +61,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 			OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op);
 			
 			
-			// the all-reduce has no combiner, when the DOP of the input is one
+			// the all-reduce has no combiner, when the parallelism of the input is one
 			
 			SourcePlanNode sourceNode = resolver.getNode("source");
 			SingleInputPlanNode reduceNode = resolver.getNode("reducer");
@@ -71,7 +71,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 			assertEquals(sourceNode, reduceNode.getInput().getSource());
 			assertEquals(reduceNode, sinkNode.getInput().getSource());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(1, sourceNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
 			assertEquals(1, sinkNode.getParallelism());
@@ -87,7 +87,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 	public void testAllReduceWithCombiner() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Long> data = env.generateSequence(1, 8000000).name("source");
 			
@@ -121,7 +121,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 			assertEquals(DriverStrategy.ALL_REDUCE, reduceNode.getDriverStrategy());
 			assertEquals(DriverStrategy.ALL_REDUCE, combineNode.getDriverStrategy());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(8, sourceNode.getParallelism());
 			assertEquals(8, combineNode.getParallelism());
 			assertEquals(1, reduceNode.getParallelism());
@@ -138,7 +138,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 	public void testGroupedReduceWithFieldPositionKey() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -179,7 +179,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 			assertEquals(new FieldList(1), combineNode.getKeys(0));
 			assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, combineNode.getParallelism());
 			assertEquals(8, reduceNode.getParallelism());
@@ -196,7 +196,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 	public void testGroupedReduceWithSelectorFunctionKey() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(8);
+			env.setParallelism(8);
 			
 			DataSet<Tuple2<String, Double>> data = env.readCsvFile("file:///will/never/be/read").types(String.class, Double.class)
 				.name("source").setParallelism(6);
@@ -243,7 +243,7 @@ public class ReduceCompilationTest extends CompilerTestBase implements java.io.S
 			assertEquals(new FieldList(0), combineNode.getKeys(0));
 			assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys());
 			
-			// check DOP
+			// check parallelism
 			assertEquals(6, sourceNode.getParallelism());
 			assertEquals(6, keyExtractor.getParallelism());
 			assertEquals(6, combineNode.getParallelism());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java
index 8720aa7..f1c2233 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java
@@ -196,7 +196,7 @@ public class WorksetIterationsJavaApiCompilerTest extends CompilerTestBase {
 	public void testRejectPlanIfSolutionSetKeysAndJoinKeysDontMatch() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+			env.setParallelism(DEFAULT_PARALLELISM);
 			
 			@SuppressWarnings("unchecked")
 			DataSet<Tuple3<Long, Long, Long>> solutionSetInput = env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Solution Set");
@@ -245,7 +245,7 @@ public class WorksetIterationsJavaApiCompilerTest extends CompilerTestBase {
 	private Plan getJavaTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) {
 		
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+		env.setParallelism(DEFAULT_PARALLELISM);
 		
 		@SuppressWarnings("unchecked")
 		DataSet<Tuple3<Long, Long, Long>> solutionSetInput = env.fromElements(new Tuple3<Long, Long, Long>(1L, 2L, 3L)).name("Solution Set");

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 91d01a2..d0615b3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -71,7 +71,7 @@ import static akka.dispatch.Futures.future;
  *         The ExecutionJobVertex is identified inside the graph by the {@link JobVertexID}, which it takes
  *         from the JobGraph's corresponding JobVertex.</li>
  *     <li>The {@link ExecutionVertex} represents one parallel subtask. For each ExecutionJobVertex, there are
- *         as many ExecutionVertices as the degree of parallelism. The ExecutionVertex is identified by
+ *         as many ExecutionVertices as the parallelism. The ExecutionVertex is identified by
  *         the ExecutionJobVertex and the number of the parallel subtask</li>
  *     <li>The {@link Execution} is one attempt to execute a ExecutionVertex. There may be multiple Executions
  *         for the ExecutionVertex, in case of a failure, or in the case where some data needs to be recomputed

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index 8816a69..c948155 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -194,22 +194,22 @@ public class AbstractJobVertex implements java.io.Serializable {
 	}
 	
 	/**
-	 * Gets the degree of parallelism of the task.
+	 * Gets the parallelism of the task.
 	 * 
-	 * @return The degree of parallelism of the task.
+	 * @return The parallelism of the task.
 	 */
 	public int getParallelism() {
 		return parallelism;
 	}
 
 	/**
-	 * Sets the degree of parallelism for the task.
+	 * Sets the parallelism for the task.
 	 * 
-	 * @param parallelism The degree of parallelism for the task.
+	 * @param parallelism The parallelism for the task.
 	 */
 	public void setParallelism(int parallelism) {
 		if (parallelism < 1) {
-			throw new IllegalArgumentException("The degree of parallelism must be at least one.");
+			throw new IllegalArgumentException("The parallelism must be at least one.");
 		}
 		this.parallelism = parallelism;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java
index fb32a6e..47b1b96 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/ShipStrategyType.java
@@ -36,7 +36,7 @@ public enum ShipStrategyType {
 	FORWARD(false, false),
 	
 	/**
-	 * Repartitioning the data randomly, typically when the degree of parallelism between two nodes changes.
+	 * Repartitioning the data randomly, typically when the parallelism between two nodes changes.
 	 */
 	PARTITION_RANDOM(true, false),
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java
index dfe6b50..f0001a9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java
@@ -86,7 +86,7 @@ public class LocalInputSplitsTest {
 				new TestLocatableInputSplit(3, "host3")
 		};
 		
-		// This should fail with an exception, since the DOP of 2 does not
+		// This should fail with an exception, since the parallelism of 2 does not
 		// support strictly local assignment onto 3 hosts
 		try {
 			runTests(numHosts, slotsPerHost, parallelism, splits);


[7/9] flink git commit: [FLINK-1679] use a consistent name for parallelism

Posted by mx...@apache.org.
[FLINK-1679] use a consistent name for parallelism

* rename occurrences of degree of parallelism to parallelism

* [Dd]egree[ -]of[ -]parallelism -> [pP]arallelism
* (DOP|dop) -> [pP]arallelism
* paraDegree -> parallelism
* degree-of-parallelism -> parallelism
* DEGREE_OF_PARALLELISM -> PARALLELISM


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/cf84bca1
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/cf84bca1
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/cf84bca1

Branch: refs/heads/master
Commit: cf84bca124ac34875ef3e559abbbf509c1218542
Parents: d994d2e
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 18 10:44:43 2015 +0100
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Mar 23 09:03:50 2015 +0100

----------------------------------------------------------------------
 docs/cluster_setup.md                           |   2 +-
 docs/config.md                                  |   4 +-
 docs/gelly_guide.md                             |   6 +-
 docs/internal_job_scheduling.md                 |   2 +-
 docs/programming_guide.md                       |  28 +-
 docs/setup_quickstart.md                        |   2 +-
 docs/streaming_guide.md                         |   6 +-
 docs/yarn_setup.md                              |   2 +-
 .../org/apache/flink/client/CliFrontend.java    |  16 +-
 .../flink/client/FlinkYarnSessionCli.java       |   4 +-
 .../org/apache/flink/client/program/Client.java |   2 +-
 .../client/program/ContextEnvironment.java      |   8 +-
 .../flink/api/common/ExecutionConfig.java       |   2 +-
 .../java/org/apache/flink/api/common/Plan.java  |  22 +-
 .../api/common/functions/RuntimeContext.java    |   8 +-
 .../flink/api/common/io/FinalizeOnMaster.java   |   2 +-
 .../flink/api/common/io/InitializeOnMaster.java |   2 +-
 .../flink/api/common/io/NonParallelInput.java   |   2 +-
 .../api/common/io/ReplicatingInputFormat.java   |   4 +-
 .../flink/configuration/ConfigConstants.java    |   8 +-
 .../api/common/io/SequentialFormatTestBase.java |  34 +-
 .../api/common/io/SerializedFormatTest.java     |   4 +-
 flink-dist/src/main/resources/flink-conf.yaml   |   2 +-
 .../java/org/apache/flink/api/java/DataSet.java |   2 +-
 .../flink/api/java/ExecutionEnvironment.java    |  64 ++--
 .../apache/flink/api/java/LocalEnvironment.java |   6 +-
 .../flink/api/java/RemoteEnvironment.java       |  10 +-
 .../flink/api/java/SortPartitionOperator.java   |   4 +-
 .../flink/api/java/io/PrintingOutputFormat.java |   2 +-
 .../api/java/operators/AggregateOperator.java   |   6 +-
 .../api/java/operators/CoGroupOperator.java     |  20 +-
 .../flink/api/java/operators/CrossOperator.java |   2 +-
 .../flink/api/java/operators/DataSink.java      |  32 +-
 .../flink/api/java/operators/DataSource.java    |   4 +-
 .../api/java/operators/DeltaIteration.java      |   8 +-
 .../api/java/operators/DistinctOperator.java    |   6 +-
 .../api/java/operators/FilterOperator.java      |  10 +-
 .../api/java/operators/FlatMapOperator.java     |  10 +-
 .../java/operators/GroupCombineOperator.java    |  14 +-
 .../api/java/operators/GroupReduceOperator.java |  14 +-
 .../flink/api/java/operators/JoinOperator.java  |  30 +-
 .../flink/api/java/operators/MapOperator.java   |  10 +-
 .../java/operators/MapPartitionOperator.java    |  10 +-
 .../flink/api/java/operators/Operator.java      |  20 +-
 .../api/java/operators/OperatorTranslation.java |   2 +-
 .../api/java/operators/PartitionOperator.java   |  12 +-
 .../api/java/operators/ProjectOperator.java     |   4 +-
 .../api/java/operators/ReduceOperator.java      |  16 +-
 .../api/java/io/TypeSerializerFormatTest.java   |   4 +-
 .../translation/AggregateTranslationTest.java   |   6 +-
 .../DeltaIterationTranslationTest.java          |  12 +-
 .../translation/DistinctTranslationTest.java    |  42 +--
 .../translation/ReduceTranslationTests.java     |  28 +-
 .../org/apache/flink/optimizer/Optimizer.java   |  30 +-
 .../flink/optimizer/dag/BinaryUnionNode.java    |  18 +-
 .../flink/optimizer/dag/BulkIterationNode.java  |   6 +-
 .../flink/optimizer/dag/DataSinkNode.java       |   4 +-
 .../flink/optimizer/dag/DataSourceNode.java     |   6 +-
 .../flink/optimizer/dag/GroupCombineNode.java   |   2 +-
 .../flink/optimizer/dag/GroupReduceNode.java    |   2 +-
 .../flink/optimizer/dag/OptimizerNode.java      |   6 +-
 .../apache/flink/optimizer/dag/ReduceNode.java  |   2 +-
 .../flink/optimizer/dag/SingleInputNode.java    |  21 +-
 .../apache/flink/optimizer/dag/SinkJoiner.java  |   2 +-
 .../flink/optimizer/dag/TwoInputNode.java       |  22 +-
 .../optimizer/dag/WorksetIterationNode.java     |   8 +-
 .../RequestedGlobalProperties.java              |   2 +-
 .../AllGroupWithPartialPreGroupProperties.java  |   4 +-
 .../operators/AllReduceProperties.java          |   4 +-
 .../operators/GroupCombineProperties.java       |   2 +-
 .../GroupReduceWithCombineProperties.java       |   4 +-
 .../operators/PartialGroupProperties.java       |   4 +-
 .../optimizer/operators/ReduceProperties.java   |   4 +-
 .../apache/flink/optimizer/plan/Channel.java    |   2 +-
 .../optimizer/plan/SinkJoinerPlanNode.java      |   2 +-
 .../plandump/PlanJSONDumpGenerator.java         |   2 +-
 .../plantranslate/JobGraphGenerator.java        |  24 +-
 .../traversals/GraphCreatingVisitor.java        |  24 +-
 .../optimizer/BranchingPlansCompilerTest.java   |  20 +-
 .../CachedMatchStrategyCompilerTest.java        |   4 +-
 .../flink/optimizer/CompilerTestBase.java       |   4 +-
 .../apache/flink/optimizer/DOPChangeTest.java   | 347 -------------------
 .../optimizer/DistinctCompilationTest.java      |  12 +-
 .../flink/optimizer/IterationsCompilerTest.java |  10 +-
 .../flink/optimizer/ParallelismChangeTest.java  | 347 +++++++++++++++++++
 .../flink/optimizer/PipelineBreakerTest.java    |  14 +-
 .../flink/optimizer/PropertyDataSourceTest.java |  48 +--
 .../optimizer/ReplicatingDataSourceTest.java    |  32 +-
 .../custompartition/CustomPartitioningTest.java |  12 +-
 .../java/DistinctAndGroupingOptimizerTest.java  |   4 +-
 .../java/GroupReduceCompilationTest.java        |  26 +-
 .../optimizer/java/IterationCompilerTest.java   |   8 +-
 .../optimizer/java/ReduceCompilationTest.java   |  18 +-
 .../WorksetIterationsJavaApiCompilerTest.java   |   4 +-
 .../runtime/executiongraph/ExecutionGraph.java  |   2 +-
 .../runtime/jobgraph/AbstractJobVertex.java     |  10 +-
 .../operators/shipping/ShipStrategyType.java    |   2 +-
 .../executiongraph/LocalInputSplitsTest.java    |   2 +-
 .../scala/operators/ScalaAggregateOperator.java |   8 +-
 .../org/apache/flink/api/scala/DataSet.scala    |  14 +-
 .../flink/api/scala/ExecutionEnvironment.scala  |  34 +-
 .../graph/spargel/VertexCentricIteration.java   |   8 +-
 .../operations/DegreesWithExceptionITCase.java  |  10 +-
 .../mapred/HadoopMapredITCase.java              |   2 +-
 .../mapreduce/HadoopInputOutputITCase.java      |   2 +-
 .../flink/ml/feature/PolynomialBaseITCase.scala |   6 +-
 .../flink/ml/recommendation/ALSITCase.scala     |   2 +-
 .../MultipleLinearRegressionITCase.scala        |   4 +-
 .../spargel/java/VertexCentricIteration.java    |   8 +-
 .../flink/spargel/java/SpargelCompilerTest.java |   4 +-
 .../spargel/java/SpargelTranslationTest.java    |  12 +-
 .../connectors/kafka/KafkaConsumerExample.java  |   2 +-
 .../connectors/kafka/KafkaProducerExample.java  |   2 +-
 .../kafka/KafkaSimpleConsumerExample.java       |   2 +-
 .../streaming/connectors/kafka/KafkaITCase.java |   2 +-
 .../api/datastream/ConnectedDataStream.java     |   2 +-
 .../streaming/api/datastream/DataStream.java    |  10 +-
 .../api/datastream/DataStreamSource.java        |   6 +-
 .../api/datastream/DiscretizedStream.java       |   2 +-
 .../datastream/SingleOutputStreamOperator.java  |  16 +-
 .../api/datastream/WindowedDataStream.java      |   4 +-
 .../api/environment/LocalStreamEnvironment.java |   4 +-
 .../environment/RemoteStreamEnvironment.java    |   4 +-
 .../environment/StreamContextEnvironment.java   |  12 +-
 .../environment/StreamExecutionEnvironment.java |  54 +--
 .../api/environment/StreamPlanEnvironment.java  |  12 +-
 .../api/function/sink/PrintSinkFunction.java    |   2 +-
 .../api/function/source/SourceFunction.java     |   2 +-
 .../flink/streaming/util/ClusterUtil.java       |   6 +-
 .../util/StreamingProgramTestBase.java          |  18 +-
 .../streaming/util/TestStreamEnvironment.java   |  10 +-
 .../flink/streaming/api/scala/DataStream.scala  |   8 +-
 .../api/scala/StreamExecutionEnvironment.scala  |  26 +-
 .../java/org/apache/flink/tachyon/HDFSTest.java |   2 +-
 .../tachyon/TachyonFileSystemWrapperTest.java   |   4 +-
 .../test/compiler/util/CompilerTestBase.java    |   4 +-
 .../flink/test/util/JavaProgramTestBase.java    |  22 +-
 .../flink/test/util/RecordAPITestBase.java      |   4 +-
 .../apache/flink/test/util/TestEnvironment.java |   4 +-
 .../test/accumulators/AccumulatorITCase.java    |   2 +-
 .../broadcastvars/BroadcastBranchingITCase.java |   2 +-
 .../BroadcastVarInitializationITCase.java       |   2 +-
 .../BroadcastVarsNepheleITCase.java             |   6 +-
 .../KMeansIterativeNepheleITCase.java           |  12 +-
 .../test/cancelling/MapCancelingITCase.java     |  12 +-
 .../cancelling/MatchJoinCancelingITCase.java    |  18 +-
 .../clients/examples/LocalExecutorITCase.java   |   6 +-
 .../exampleScalaPrograms/WordCountITCase.java   |   2 +-
 .../test/failingPrograms/TaskFailureITCase.java |   6 +-
 .../CoGroupConnectedComponentsITCase.java       |   4 +-
 .../iterative/ConnectedComponentsITCase.java    |   4 +-
 ...ectedComponentsWithDeferredUpdateITCase.java |   4 +-
 ...tedComponentsWithSolutionSetFirstITCase.java |   4 +-
 .../test/iterative/DanglingPageRankITCase.java  |   2 +-
 ...terationNotDependingOnSolutionSetITCase.java |   2 +-
 .../test/iterative/DeltaPageRankITCase.java     |   2 +-
 .../DependencyConnectedComponentsITCase.java    |   6 +-
 ...IterationTerminationWithTerminationTail.java |   4 +-
 .../IterationTerminationWithTwoTails.java       |   4 +-
 .../IterationWithAllReducerITCase.java          |   2 +-
 .../iterative/IterationWithChainingITCase.java  |   6 +-
 .../test/iterative/IterativeKMeansITCase.java   |   4 +-
 .../flink/test/iterative/KMeansITCase.java      |   4 +-
 .../flink/test/iterative/LineRankITCase.java    |   4 +-
 .../flink/test/iterative/PageRankITCase.java    |   2 +-
 .../aggregators/AggregatorsITCase.java          |  12 +-
 ...nentsWithParametrizableAggregatorITCase.java |   4 +-
 ...entsWithParametrizableConvergenceITCase.java |   4 +-
 .../ConnectedComponentsNepheleITCase.java       |  14 +-
 .../nephele/DanglingPageRankNepheleITCase.java  |   4 +-
 ...nglingPageRankWithCombinerNepheleITCase.java |   4 +-
 .../IterationWithChainingNepheleITCase.java     |   4 +-
 .../test/iterative/nephele/JobGraphUtils.java   |   8 +-
 .../CustomCompensatableDanglingPageRank.java    |  20 +-
 ...mpensatableDanglingPageRankWithCombiner.java |  20 +-
 .../CompensatableDanglingPageRank.java          |  20 +-
 .../test/javaApiOperators/DataSinkITCase.java   |  22 +-
 .../javaApiOperators/GroupCombineITCase.java    |   4 +-
 .../javaApiOperators/GroupReduceITCase.java     |  32 +-
 .../flink/test/javaApiOperators/JoinITCase.java |  14 +-
 .../test/javaApiOperators/PartitionITCase.java  |  12 +-
 .../javaApiOperators/SortPartitionITCase.java   |  20 +-
 .../flink/test/misc/AutoParallelismITCase.java  |   2 +-
 .../test/misc/CustomPartitioningITCase.java     |   2 +-
 .../flink/test/misc/NullValuesITCase.java       |   2 +-
 .../flink/test/operators/CoGroupITCase.java     |   8 +-
 .../flink/test/operators/CrossITCase.java       |   8 +-
 .../apache/flink/test/operators/JoinITCase.java |   8 +-
 .../apache/flink/test/operators/MapITCase.java  |   6 +-
 .../flink/test/operators/ReduceITCase.java      |   6 +-
 .../flink/test/operators/UnionITCase.java       |   8 +-
 .../flink/test/operators/UnionSinkITCase.java   |   4 +-
 .../recordJobTests/CollectionSourceTest.java    |   6 +-
 .../ComputeEdgeDegreesITCase.java               |   4 +-
 .../EnumTrianglesOnEdgesWithDegreesITCase.java  |   4 +-
 .../recordJobTests/EnumTrianglesRDFITCase.java  |   4 +-
 .../recordJobTests/GlobalSortingITCase.java     |   8 +-
 .../GlobalSortingMixedOrderITCase.java          |   4 +-
 .../recordJobTests/GroupOrderReduceITCase.java  |   8 +-
 .../test/recordJobTests/PairwiseSPITCase.java   |   4 +-
 .../test/recordJobTests/TPCHQuery10ITCase.java  |   2 +-
 .../test/recordJobTests/TPCHQuery3ITCase.java   |   6 +-
 .../TPCHQuery3WithUnionITCase.java              |   4 +-
 .../test/recordJobTests/TPCHQuery4ITCase.java   |   4 +-
 .../test/recordJobTests/TPCHQuery9ITCase.java   |   4 +-
 .../recordJobTests/TPCHQueryAsterixITCase.java  |   4 +-
 .../test/recordJobTests/TeraSortITCase.java     |   4 +-
 .../recordJobTests/WebLogAnalysisITCase.java    |   4 +-
 .../test/recordJobTests/WordCountITCase.java    |   4 +-
 .../WordCountUnionReduceITCase.java             |   4 +-
 .../test/recordJobs/graph/DanglingPageRank.java |   8 +-
 .../flink/test/recordJobs/graph/PairwiseSP.java |   8 +-
 .../test/recordJobs/graph/SimplePageRank.java   |   8 +-
 .../test/recordJobs/kmeans/KMeansBroadcast.java |   4 +-
 .../test/recordJobs/kmeans/KMeansCross.java     |   2 +-
 .../recordJobs/relational/MergeOnlyJoin.java    |   4 +-
 .../test/recordJobs/relational/TPCHQuery1.java  |  14 +-
 .../test/recordJobs/relational/TPCHQuery10.java |   6 +-
 .../test/recordJobs/relational/TPCHQuery4.java  |  20 +-
 .../test/recordJobs/relational/TPCHQuery9.java  |  10 +-
 .../recordJobs/relational/TPCHQueryAsterix.java |  10 +-
 .../recordJobs/relational/WebLogAnalysis.java   |   2 +-
 .../flink/test/recordJobs/sort/TeraSort.java    |   4 +-
 .../ProcessFailureBatchRecoveryITCase.java      |   2 +-
 .../test/recovery/SimpleRecoveryITCase.java     |   8 +-
 .../TaskManagerFailureRecoveryITCase.java       |   2 +-
 ...ConsumePipelinedAndBlockingResultITCase.java |   2 +-
 .../flink/test/util/testjar/KMeansForTest.java  |   2 +-
 .../api/scala/operators/GroupReduceITCase.scala |  28 +-
 .../flink/api/scala/operators/JoinITCase.scala  |  12 +-
 .../api/scala/operators/PartitionITCase.scala   |  24 +-
 .../translation/AggregateTranslationTest.scala  |   6 +-
 .../translation/CustomPartitioningTest.scala    |  10 +-
 .../DeltaIterationTranslationTest.scala         |  12 +-
 .../translation/ReduceTranslationTest.scala     |  22 +-
 .../apache/flink/yarn/ApplicationMaster.scala   |   2 +-
 236 files changed, 1395 insertions(+), 1392 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/cluster_setup.md
----------------------------------------------------------------------
diff --git a/docs/cluster_setup.md b/docs/cluster_setup.md
index d425be8..2ee9f3c 100644
--- a/docs/cluster_setup.md
+++ b/docs/cluster_setup.md
@@ -323,7 +323,7 @@ In particular,
 
  * the amount of available memory per TaskManager (`taskmanager.heap.mb`), 
  * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`),
- * the total number of CPUs in the cluster (`parallelization.degree.default`) and 
+ * the total number of CPUs in the cluster (`parallelism.default`) and
  * the temporary directories (`taskmanager.tmp.dirs`)
 
 are very important configuration values.

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/config.md
----------------------------------------------------------------------
diff --git a/docs/config.md b/docs/config.md
index b8cf06a..09a01d9 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -80,8 +80,8 @@ This value is typically proportional to the number of physical CPU cores that
 the TaskManager's machine has (e.g., equal to the number of cores, or half the
 number of cores). [More about task slots](config.html#configuring-taskmanager-processing-slots).
 
-- `parallelization.degree.default`: The default degree of parallelism to use for
-programs that have no degree of parallelism specified. (DEFAULT: 1). For
+- `parallelism.default`: The default parallelism to use for
+programs that have no parallelism specified. (DEFAULT: 1). For
 setups that have no concurrent jobs running, setting this value to
 NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all
 available execution resources for the program's execution.

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/gelly_guide.md
----------------------------------------------------------------------
diff --git a/docs/gelly_guide.md b/docs/gelly_guide.md
index 8058e0a..32c076b 100644
--- a/docs/gelly_guide.md
+++ b/docs/gelly_guide.md
@@ -344,7 +344,7 @@ Gelly wraps Flink's [Spargel API](spargel_guide.html) to provide methods for ver
 Like in Spargel, the user only needs to implement two functions: a `VertexUpdateFunction`, which defines how a vertex will update its value
 based on the received messages and a `MessagingFunction`, which allows a vertex to send out messages for the next superstep.
 These functions are given as parameters to Gelly's `createVertexCentricIteration`, which returns a `VertexCentricIteration`. 
-The user can configure this iteration (set the name, the degree of parallelism, aggregators, etc.) and then run the computation, using the `runVertexCentricIteration` method:
+The user can configure this iteration (set the name, the parallelism, aggregators, etc.) and then run the computation, using the `runVertexCentricIteration` method:
 
 {% highlight java %}
 Graph<Long, Double, Double> graph = ...
@@ -357,8 +357,8 @@ VertexCentricIteration<Long, Double, Double, Double> iteration =
 // set the iteration name
 iteration.setName("Single Source Shortest Paths");
 
-// set the degree of parallelism
-iteration.setDegreeOfParallelism(16);
+// set the parallelism
+iteration.setParallelism(16);
 
 // run the computation
 graph.runVertexCentricIteration(iteration);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/internal_job_scheduling.md
----------------------------------------------------------------------
diff --git a/docs/internal_job_scheduling.md b/docs/internal_job_scheduling.md
index 0d08f76..2c55c01 100644
--- a/docs/internal_job_scheduling.md
+++ b/docs/internal_job_scheduling.md
@@ -57,7 +57,7 @@ and reacts to finished tasks or execution failures.
 The JobManager receives the {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ "JobGraph" %},
 which is a representation of the data flow consisting of operators ({% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java "JobVertex" %})
 and intermediate results ({% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java "IntermediateDataSet" %}).
-Each operator has properies, like the degree of parallelism and the code that it executes.
+Each operator has properies, like the parallelism and the code that it executes.
 In addition, the JobGraph has a set of attached libraries, that are neccessary to execute the code of the operators.
 
 The JobManager transforms the JobGraph into an {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ "ExecutionGraph" %}.

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/programming_guide.md
----------------------------------------------------------------------
diff --git a/docs/programming_guide.md b/docs/programming_guide.md
index e1c79ee..1750a12 100644
--- a/docs/programming_guide.md
+++ b/docs/programming_guide.md
@@ -218,10 +218,10 @@ obtain one using these static methods on class `ExecutionEnvironment`:
 getExecutionEnvironment()
 
 createLocalEnvironment()
-createLocalEnvironment(int degreeOfParallelism)
+createLocalEnvironment(int parallelism)
 
 createRemoteEnvironment(String host, int port, String... jarFiles)
-createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles)
+createRemoteEnvironment(String host, int port, int parallelism, String... jarFiles)
 {% endhighlight %}
 
 Typically, you only need to use `getExecutionEnvironment()`, since this
@@ -318,10 +318,10 @@ obtain one using these static methods on class `ExecutionEnvironment`:
 {% highlight scala %}
 def getExecutionEnvironment
 
-def createLocalEnvironment(degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors()))
+def createLocalEnvironment(parallelism: Int = Runtime.getRuntime.availableProcessors()))
 
 def createRemoteEnvironment(host: String, port: String, jarFiles: String*)
-def createRemoteEnvironment(host: String, port: String, degreeOfParallelism: Int, jarFiles: String*)
+def createRemoteEnvironment(host: String, port: String, parallelism: Int, jarFiles: String*)
 {% endhighlight %}
 
 Typically, you only need to use `getExecutionEnvironment()`, since this
@@ -2074,7 +2074,7 @@ val myLongs = env.fromCollection(longIt)
 </div>
 
 **Note:** Currently, the collection data source requires that data types and iterators implement
-`Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of
+`Serializable`. Furthermore, collection data sources can not be executed in parallel (
 parallelism = 1).
 
 [Back to top](#top)
@@ -2704,15 +2704,15 @@ Parallel Execution
 This section describes how the parallel execution of programs can be configured in Flink. A Flink
 program consists of multiple tasks (operators, data sources, and sinks). A task is split into
 several parallel instances for execution and each parallel instance processes a subset of the task's
-input data. The number of parallel instances of a task is called its *parallelism* or *degree of
-parallelism (DOP)*.
+input data. The number of parallel instances of a task is called its *parallelism*.
 
-The degree of parallelism of a task can be specified in Flink on different levels.
+
+The parallelism of a task can be specified in Flink on different levels.
 
 ### Operator Level
 
 The parallelism of an individual operator, data source, or data sink can be defined by calling its
-`setParallelism()` method.  For example, the degree of parallelism of the `Sum` operator in the
+`setParallelism()` method.  For example, the parallelism of the `Sum` operator in the
 [WordCount](#example-program) example program can be set to `5` as follows :
 
 
@@ -2749,13 +2749,13 @@ env.execute("Word Count Example")
 
 ### Execution Environment Level
 
-Flink programs are executed in the context of an [execution environmentt](#program-skeleton). An
+Flink programs are executed in the context of an [execution environment](#program-skeleton). An
 execution environment defines a default parallelism for all operators, data sources, and data sinks
 it executes. Execution environment parallelism can be overwritten by explicitly configuring the
 parallelism of an operator.
 
 The default parallelism of an execution environment can be specified by calling the
-`setDegreeOfParallelism()` method. To execute all operators, data sources, and data sinks of the
+`setParallelism()` method. To execute all operators, data sources, and data sinks of the
 [WordCount](#example-program) example program with a parallelism of `3`, set the default parallelism of the
 execution environment as follows:
 
@@ -2763,7 +2763,7 @@ execution environment as follows:
 <div data-lang="java" markdown="1">
 {% highlight java %}
 final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-env.setDegreeOfParallelism(3);
+env.setParallelism(3);
 
 DataSet<String> text = [...]
 DataSet<Tuple2<String, Integer>> wordCounts = [...]
@@ -2775,7 +2775,7 @@ env.execute("Word Count Example");
 <div data-lang="scala" markdown="1">
 {% highlight scala %}
 val env = ExecutionEnvironment.getExecutionEnvironment
-env.setDegreeOfParallelism(3)
+env.setParallelism(3)
 
 val text = [...]
 val wordCounts = text
@@ -2792,7 +2792,7 @@ env.execute("Word Count Example")
 ### System Level
 
 A system-wide default parallelism for all execution environments can be defined by setting the
-`parallelization.degree.default` property in `./conf/flink-conf.yaml`. See the
+`parallelism.default` property in `./conf/flink-conf.yaml`. See the
 [Configuration](config.html) documentation for details.
 
 [Back to top](#top)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/setup_quickstart.md b/docs/setup_quickstart.md
index e45792d..5a19d83 100644
--- a/docs/setup_quickstart.md
+++ b/docs/setup_quickstart.md
@@ -139,7 +139,7 @@ In particular,
 
  * the amount of available memory per TaskManager (`taskmanager.heap.mb`), 
  * the number of available CPUs per machine (`taskmanager.numberOfTaskSlots`),
- * the total number of CPUs in the cluster (`parallelization.degree.default`) and 
+ * the total number of CPUs in the cluster (`parallelism.default`) and
  * the temporary directories (`taskmanager.tmp.dirs`)
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
index 6d62b75..85edc16 100644
--- a/docs/streaming_guide.md
+++ b/docs/streaming_guide.md
@@ -281,7 +281,7 @@ The user is expected to connect to the outside world through the source and the
 
 The user can connect to data streams by the different implementations of `SourceFunction` using `StreamExecutionEnvironment.addSource(sourceFunction)`. In contrast with other operators, DataStreamSources have a default operator parallelism of 1.
 
-To create parallel sources the users source function needs to implement `ParallelSourceFunction` or extend `RichParallelSourceFunction` in which cases the source will have the parallelism of the environment. The degree of parallelism for ParallelSourceFunctions can be changed afterwards using `source.setParallelism(dop)`.
+To create parallel sources the users source function needs to implement `ParallelSourceFunction` or extend `RichParallelSourceFunction` in which cases the source will have the parallelism of the environment. The parallelism for ParallelSourceFunctions can be changed afterwards using `source.setParallelism(parallelism)`.
 
 There are several predefined ones similar to the ones of the batch API and some streaming specific ones like:
 
@@ -751,7 +751,7 @@ The above call would create global windows of 1000 elements group it by the firs
 Notice that here we only defined the window size once at the beginning of the transformation. This means that anything that happens afterwards (`groupBy(firstKey).mapWindow(…).groupBy(secondKey).reduceWindow(…)`) happens inside the 1000 element windows. Of course the mapWindow might reduce the number of elements but the key idea is that each transformation still corresponds to the same 1000 elements in the original stream.
 
 #### Global vs local discretisation
-By default all window discretisation calls (`dataStream.window(…)`) define global windows meaning that a global window of count 100 will contain the last 100 elements arrived at the discretisation operator in order. In most cases (except for Time) this means that the operator doing the actual discretisation needs to have a degree of parallelism of 1 to be able to correctly execute the discretisation logic.
+By default all window discretisation calls (`dataStream.window(…)`) define global windows meaning that a global window of count 100 will contain the last 100 elements arrived at the discretisation operator in order. In most cases (except for Time) this means that the operator doing the actual discretisation needs to have a parallelism of 1 to be able to correctly execute the discretisation logic.
 
 Sometimes it is sufficient to create local discretisations, which allows the discretiser to run in parallel and apply the given discretisation logic at every discretiser instance. To allow local discretisation use the `local()` method of the windowed data stream.
 
@@ -1109,7 +1109,7 @@ Operator Settings
 
 ### Parallelism
 
-Setting parallelism for operators works exactly the same way as in the batch Flink API. The user can control the number of parallel instances created for each operator by calling the `operator.setParallelism(dop)` method.
+Setting parallelism for operators works exactly the same way as in the batch Flink API. The user can control the number of parallel instances created for each operator by calling the `operator.setParallelism(parallelism)` method.
 
 ### Buffer timeout
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/docs/yarn_setup.md
----------------------------------------------------------------------
diff --git a/docs/yarn_setup.md b/docs/yarn_setup.md
index d65cf7e..4da8a05 100644
--- a/docs/yarn_setup.md
+++ b/docs/yarn_setup.md
@@ -112,7 +112,7 @@ Please note that the Client requires the `YARN_CONF_DIR` or `HADOOP_CONF_DIR` en
 
 The system will use the configuration in `conf/flink-config.yaml`. Please follow our [configuration guide](config.html) if you want to change something. 
 
-Flink on YARN will overwrite the following configuration parameters `jobmanager.rpc.address` (because the JobManager is always allocated at different machines), `taskmanager.tmp.dirs` (we are using the tmp directories given by YARN) and `parallelization.degree.default` if the number of slots has been specified.
+Flink on YARN will overwrite the following configuration parameters `jobmanager.rpc.address` (because the JobManager is always allocated at different machines), `taskmanager.tmp.dirs` (we are using the tmp directories given by YARN) and `parallelism.default` if the number of slots has been specified.
 
 If you don't want to change the configuration file to set configuration parameters, there is the option to pass dynamic properties via the `-D` flag. So you can pass parameters this way: `-Dfs.overwrite-files=true -Dtaskmanager.network.numberOfBuffers=16368`.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index 1d9d956..dd2a0ba 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -102,7 +102,7 @@ public class CliFrontend {
 	// YARN-session related constants
 	public static final String YARN_PROPERTIES_FILE = ".yarn-properties";
 	public static final String YARN_PROPERTIES_JOBMANAGER_KEY = "jobManager";
-	public static final String YARN_PROPERTIES_DOP = "degreeOfParallelism";
+	public static final String YARN_PROPERTIES_PARALLELISM = "parallelism";
 	public static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString";
 
 	public static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split()
@@ -175,18 +175,18 @@ public class CliFrontend {
 				throw new Exception("Cannot read the YARN properties file", e);
 			}
 
-			// configure the default degree of parallelism from YARN
-			String propDegree = yarnProperties.getProperty(YARN_PROPERTIES_DOP);
-			if (propDegree != null) { // maybe the property is not set
+			// configure the default parallelism from YARN
+			String propParallelism = yarnProperties.getProperty(YARN_PROPERTIES_PARALLELISM);
+			if (propParallelism != null) { // maybe the property is not set
 				try {
-					int paraDegree = Integer.parseInt(propDegree);
-					this.config.setInteger(ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY, paraDegree);
+					int parallelism = Integer.parseInt(propParallelism);
+					this.config.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism);
 
-					logAndSysout("YARN properties set default parallelism to " + paraDegree);
+					logAndSysout("YARN properties set default parallelism to " + parallelism);
 				}
 				catch (NumberFormatException e) {
 					throw new Exception("Error while parsing the YARN properties: " +
-							"Property " + YARN_PROPERTIES_DOP + " is not an integer.");
+							"Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer.");
 				}
 			}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java
index 340b878..25f31e3 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java
@@ -408,9 +408,9 @@ public class FlinkYarnSessionCli {
 			Properties yarnProps = new Properties();
 			yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_JOBMANAGER_KEY, jobManagerAddress);
 			if (flinkYarnClient.getTaskManagerSlots() != -1) {
-				String degreeOfParallelism =
+				String parallelism =
 						Integer.toString(flinkYarnClient.getTaskManagerSlots() * flinkYarnClient.getTaskManagerCount());
-				yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_DOP, degreeOfParallelism);
+				yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_PARALLELISM, parallelism);
 			}
 			// add dynamic properties
 			if (flinkYarnClient.getDynamicPropertiesEncoded() != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
index 4e593c8..9a578bc 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
@@ -145,7 +145,7 @@ public class Client {
 			// temporary hack to support the optimizer plan preview
 			OptimizerPlanEnvironment env = new OptimizerPlanEnvironment(this.compiler);
 			if (parallelism > 0) {
-				env.setDegreeOfParallelism(parallelism);
+				env.setParallelism(parallelism);
 			}
 			env.setAsContext();
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
index c1fe9b4..8d5fe17 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
@@ -52,14 +52,14 @@ public class ContextEnvironment extends ExecutionEnvironment {
 		Plan p = createProgramPlan(jobName);
 		JobWithJars toRun = new JobWithJars(p, this.jarFilesToAttach, this.userCodeClassLoader);
 		
-		return this.client.run(toRun, getDegreeOfParallelism(), true);
+		return this.client.run(toRun, getParallelism(), true);
 	}
 
 	@Override
 	public String getExecutionPlan() throws Exception {
 		Plan p = createProgramPlan("unnamed job");
 		
-		OptimizedPlan op = (OptimizedPlan) this.client.getOptimizedPlan(p, getDegreeOfParallelism());
+		OptimizedPlan op = (OptimizedPlan) this.client.getOptimizedPlan(p, getParallelism());
 		
 		PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
 		return gen.getOptimizerPlanAsJSON(op);
@@ -68,7 +68,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 	
 	@Override
 	public String toString() {
-		return "Context Environment (DOP = " + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism())
+		return "Context Environment (parallelism = " + (getParallelism() == -1 ? "default" : getParallelism())
 				+ ") : " + getIdString();
 	}
 	
@@ -118,7 +118,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 		public ExecutionEnvironment createExecutionEnvironment() {
 			ContextEnvironment env = new ContextEnvironment(client, jarFilesToAttach, userCodeClassLoader);
 			if (defaultParallelism > 0) {
-				env.setDegreeOfParallelism(defaultParallelism);
+				env.setParallelism(defaultParallelism);
 			}
 			return env;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 a2df438..1990a2f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -52,7 +52,7 @@ public class ExecutionConfig implements Serializable {
 	public static final String CONFIG_KEY = "runtime.config";
 
 	/**
-	 * The constant to use for the degree of parallelism, if the system should use the number
+	 * The constant to use for the parallelism, if the system should use the number
 	 *  of currently available slots.
 	 */
 	public static final int PARALLELISM_AUTO_MAX = Integer.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/main/java/org/apache/flink/api/common/Plan.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java
index d5295bb..f0cb926 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java
@@ -44,7 +44,7 @@ import org.apache.flink.util.Visitor;
 
 /**
  * This class encapsulates a single job (an instantiated data flow), together with some parameters.
- * Parameters include the name and a default degree of parallelism. The job is referenced by the data sinks,
+ * Parameters include the name and a default parallelism. The job is referenced by the data sinks,
  * from which a traversal reaches all connected nodes of the job.
  */
 public class Plan implements Visitable<Operator<?>> {
@@ -102,7 +102,7 @@ public class Plan implements Visitable<Operator<?>> {
 	 *
 	 * @param sinks The collection will the sinks of the job's data flow.
 	 * @param jobName The name to display for the job.
-	 * @param defaultParallelism The default degree of parallelism for the job.
+	 * @param defaultParallelism The default parallelism for the job.
 	 */
 	public Plan(Collection<? extends GenericDataSinkBase<?>> sinks, String jobName, int defaultParallelism) {
 		this.sinks.addAll(sinks);
@@ -134,7 +134,7 @@ public class Plan implements Visitable<Operator<?>> {
 	 *
 	 * @param sink The data sink of the data flow.
 	 * @param jobName The name to display for the job.
-	 * @param defaultParallelism The default degree of parallelism for the job.
+	 * @param defaultParallelism The default parallelism for the job.
 	 */
 	public Plan(GenericDataSinkBase<?> sink, String jobName, int defaultParallelism) {
 		this(Collections.<GenericDataSinkBase<?>>singletonList(sink), jobName, defaultParallelism);
@@ -163,7 +163,7 @@ public class Plan implements Visitable<Operator<?>> {
 	 * from the given data sinks.
 	 *
 	 * @param sinks The collection will the sinks of the data flow.
-	 * @param defaultParallelism The default degree of parallelism for the job.
+	 * @param defaultParallelism The default parallelism for the job.
 	 */
 	public Plan(Collection<? extends GenericDataSinkBase<?>> sinks, int defaultParallelism) {
 		this(sinks, "Flink Job at " + Calendar.getInstance().getTime(), defaultParallelism);
@@ -190,7 +190,7 @@ public class Plan implements Visitable<Operator<?>> {
 	 * not be translated entirely.
 	 *
 	 * @param sink The data sink of the data flow.
-	 * @param defaultParallelism The default degree of parallelism for the job.
+	 * @param defaultParallelism The default parallelism for the job.
 	 */
 	public Plan(GenericDataSinkBase<?> sink, int defaultParallelism) {
 		this(sink, "Flink Job at " + Calendar.getInstance().getTime(), defaultParallelism);
@@ -240,8 +240,8 @@ public class Plan implements Visitable<Operator<?>> {
 	}
 	
 	/**
-	 * Gets the default degree of parallelism for this job. That degree is always used when an operator
-	 * is not explicitly given a degree of parallelism.
+	 * Gets the default parallelism for this job. That degree is always used when an operator
+	 * is not explicitly given a parallelism.
 	 *
 	 * @return The default parallelism for the plan.
 	 */
@@ -250,14 +250,14 @@ public class Plan implements Visitable<Operator<?>> {
 	}
 	
 	/**
-	 * Sets the default degree of parallelism for this plan. That degree is always used when an operator
-	 * is not explicitly given a degree of parallelism.
+	 * Sets the default parallelism for this plan. That degree is always used when an operator
+	 * is not explicitly given a parallelism.
 	 *
 	 * @param defaultParallelism The default parallelism for the plan.
 	 */
 	public void setDefaultParallelism(int defaultParallelism) {
 		checkArgument(defaultParallelism >= 1 || defaultParallelism == -1,
-			"The default degree of parallelism must be positive, or -1 if the system should use the globally comfigured default.");
+			"The default parallelism must be positive, or -1 if the system should use the globally comfigured default.");
 		
 		this.defaultParallelism = defaultParallelism;
 	}
@@ -364,7 +364,7 @@ public class Plan implements Visitable<Operator<?>> {
 		
 		@Override
 		public boolean preVisit(Operator<?> visitable) {
-			this.maxDop = Math.max(this.maxDop, visitable.getDegreeOfParallelism());
+			this.maxDop = Math.max(this.maxDop, visitable.getParallelism());
 			return true;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
index e9209a8..f68d2b0 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
@@ -33,7 +33,7 @@ import org.apache.flink.api.common.cache.DistributedCache;
 /**
  * A RuntimeContext contains information about the context in which functions are executed. Each parallel instance
  * of the function will have a context through which it can access static contextual information (such as 
- * the current degree of parallelism) and other constructs like accumulators and broadcast variables.
+ * the current parallelism) and other constructs like accumulators and broadcast variables.
  * <p>
  * A function can, during runtime, obtain the RuntimeContext via a call to
  * {@link AbstractRichFunction#getRuntimeContext()}.
@@ -48,14 +48,14 @@ public interface RuntimeContext {
 	String getTaskName();
 
 	/**
-	 * Gets the degree of parallelism with which the parallel task runs.
+	 * Gets the parallelism with which the parallel task runs.
 	 * 
-	 * @return The degree of parallelism with which the parallel task runs.
+	 * @return The parallelism with which the parallel task runs.
 	 */
 	int getNumberOfParallelSubtasks();
 
 	/**
-	 * Gets the number of the parallel subtask. The numbering starts from 1 and goes up to the degree-of-parallelism,
+	 * Gets the number of the parallel subtask. The numbering starts from 1 and goes up to the parallelism,
 	 * as returned by {@link #getNumberOfParallelSubtasks()}.
 	 * 
 	 * @return The number of the parallel subtask.

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java
index 6fa535c..8b787eb 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FinalizeOnMaster.java
@@ -29,7 +29,7 @@ public interface FinalizeOnMaster {
 	/**
 	 * The method is invoked on the master (JobManager) after all (parallel) instances of an OutputFormat finished.
 	 * 
-	 * @param parallelism The degree of parallelism with which the format or functions was run.
+	 * @param parallelism The parallelism with which the format or functions was run.
 	 * @throws IOException The finalization may throw exceptions, which may cause the job to abort.
 	 */
 	void finalizeGlobal(int parallelism) throws IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java b/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
index 8d391c0..943fc8d 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
@@ -31,7 +31,7 @@ public interface InitializeOnMaster {
 	/**
 	 * The method is invoked on the master (JobManager) before the distributed program execution starts.
 	 * 
-	 * @param parallelism The degree of parallelism with which the format or functions will be run.
+	 * @param parallelism The parallelism with which the format or functions will be run.
 	 * @throws IOException The initialization may throw exceptions, which may cause the job to abort.
 	 */
 	void initializeGlobal(int parallelism) throws IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java b/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
index 00a1613..26e49a7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
@@ -20,7 +20,7 @@ package org.apache.flink.api.common.io;
 
 /**
  * This interface acts as a marker for input formats for inputs which cannot be split.
- * Data sources with a non-parallel input formats are always executed with a degree-of-parallelism
+ * Data sources with a non-parallel input formats are always executed with a parallelism
  * of one.
  * 
  * @see InputFormat

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
index 0adccaf..cc25539 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputFormat.java
@@ -33,9 +33,9 @@ import java.io.IOException;
  * replicated InputFormat to each parallel instance.
  *
  * Replicated data can only be used as input for a {@link org.apache.flink.api.common.operators.base.JoinOperatorBase} or
- * {@link org.apache.flink.api.common.operators.base.CrossOperatorBase} with the same degree of parallelism as the DataSource.
+ * {@link org.apache.flink.api.common.operators.base.CrossOperatorBase} with the same parallelism as the DataSource.
  * Before being used as an input to a Join or Cross operator, replicated data might be processed in local pipelines by
- * by Map-based operators with the same degree of parallelism as the source. Map-based operators are
+ * by Map-based operators with the same parallelism as the source. Map-based operators are
  * {@link org.apache.flink.api.common.operators.base.MapOperatorBase},
  * {@link org.apache.flink.api.common.operators.base.FlatMapOperatorBase},
  * {@link org.apache.flink.api.common.operators.base.FilterOperatorBase}, and

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 4fc89c3..b472d8a 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -31,9 +31,9 @@ public final class ConfigConstants {
 	// ---------------------------- Parallelism -------------------------------
 
 	/**
-	 * The config parameter defining the default degree of parallelism for jobs.
+	 * The config parameter defining the default parallelism for jobs.
 	 */
-	public static final String DEFAULT_PARALLELIZATION_DEGREE_KEY = "parallelization.degree.default";
+	public static final String DEFAULT_PARALLELISM_KEY = "parallelism.default";
 	
 	/**
 	 * Config parameter for the number of re-tries for failed tasks. Setting this
@@ -399,9 +399,9 @@ public final class ConfigConstants {
 	// ---------------------------- Parallelism -------------------------------
 	
 	/**
-	 * The default degree of parallelism for operations.
+	 * The default parallelism for operations.
 	 */
-	public static final int DEFAULT_PARALLELIZATION_DEGREE = 1;
+	public static final int DEFAULT_PARALLELISM = 1;
 	
 	/**
 	 * The default number of execution retries.

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java
index 8c4e090..3e4697a 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java
@@ -56,7 +56,7 @@ public abstract class SequentialFormatTestBase<T> {
 
 	protected long blockSize;
 
-	private int degreeOfParallelism;
+	private int parallelism;
 
 	private int[] rawDataSizes;
 
@@ -65,11 +65,11 @@ public abstract class SequentialFormatTestBase<T> {
 	/**
 	 * Initializes SequentialFormatTest.
 	 */
-	public SequentialFormatTestBase(int numberOfTuples, long blockSize, int degreeOfParallelism) {
+	public SequentialFormatTestBase(int numberOfTuples, long blockSize, int parallelism) {
 		this.numberOfTuples = numberOfTuples;
 		this.blockSize = blockSize;
-		this.degreeOfParallelism = degreeOfParallelism;
-		this.rawDataSizes = new int[degreeOfParallelism];
+		this.parallelism = parallelism;
+		this.rawDataSizes = new int[parallelism];
 	}
 
 	/**
@@ -78,7 +78,7 @@ public abstract class SequentialFormatTestBase<T> {
 	@Before
 	public void calcRawDataSize() throws IOException {
 		int recordIndex = 0;
-		for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) {
+		for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) {
 			ByteCounter byteCounter = new ByteCounter();
 			DataOutputStream out = new DataOutputStream(byteCounter);
 			for (int fileCount = 0; fileCount < this.getNumberOfTuplesPerFile(fileIndex); fileCount++, recordIndex++) {
@@ -98,7 +98,7 @@ public abstract class SequentialFormatTestBase<T> {
 		Arrays.sort(inputSplits, new InputSplitSorter());
 
 		int splitIndex = 0;
-		for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) {
+		for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) {
 			List<FileInputSplit> sameFileSplits = new ArrayList<FileInputSplit>();
 			Path lastPath = inputSplits[splitIndex].getPath();
 			for (; splitIndex < inputSplits.length; splitIndex++) {
@@ -178,7 +178,7 @@ public abstract class SequentialFormatTestBase<T> {
 		this.tempFile.deleteOnExit();
 		Configuration configuration = new Configuration();
 		configuration.setLong(BinaryOutputFormat.BLOCK_SIZE_PARAMETER_KEY, this.blockSize);
-		if (this.degreeOfParallelism == 1) {
+		if (this.parallelism == 1) {
 			BinaryOutputFormat<T> output = createOutputFormat(this.tempFile.toURI().toString(),
 					configuration);
 			for (int index = 0; index < this.numberOfTuples; index++) {
@@ -189,7 +189,7 @@ public abstract class SequentialFormatTestBase<T> {
 			this.tempFile.delete();
 			this.tempFile.mkdir();
 			int recordIndex = 0;
-			for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) {
+			for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) {
 				BinaryOutputFormat<T> output = createOutputFormat(this.tempFile.toURI() + "/" +
 						(fileIndex+1), configuration);
 				for (int fileCount = 0; fileCount < this.getNumberOfTuplesPerFile(fileIndex); fileCount++, recordIndex++) {
@@ -201,7 +201,7 @@ public abstract class SequentialFormatTestBase<T> {
 	}
 
 	private int getNumberOfTuplesPerFile(int fileIndex) {
-		return this.numberOfTuples / this.degreeOfParallelism;
+		return this.numberOfTuples / this.parallelism;
 	}
 
 	/**
@@ -211,7 +211,7 @@ public abstract class SequentialFormatTestBase<T> {
 	public void checkLength() {
 		File[] files = this.tempFile.isDirectory() ? this.tempFile.listFiles() : new File[] { this.tempFile };
 		Arrays.sort(files);
-		for (int fileIndex = 0; fileIndex < this.degreeOfParallelism; fileIndex++) {
+		for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) {
 			long lastBlockLength = this.rawDataSizes[fileIndex] % (this.blockSize - getInfoSize());
 			long expectedLength =
 				(this.getExpectedBlockCount(fileIndex) - 1) * this.blockSize + getInfoSize() +
@@ -252,13 +252,13 @@ public abstract class SequentialFormatTestBase<T> {
 	@Parameters
 	public static List<Object[]> getParameters() {
 		ArrayList<Object[]> params = new ArrayList<Object[]>();
-		for (int dop = 1; dop <= 2; dop++) {
-			// numberOfTuples, blockSize, dop
-			params.add(new Object[] { 100, BinaryOutputFormat.NATIVE_BLOCK_SIZE, dop });
-			params.add(new Object[] { 100, 1000, dop });
-			params.add(new Object[] { 100, 1 << 20, dop });
-			params.add(new Object[] { 10000, 1000, dop });
-			params.add(new Object[] { 10000, 1 << 20, dop });
+		for (int parallelism = 1; parallelism <= 2; parallelism++) {
+			// numberOfTuples, blockSize, parallelism
+			params.add(new Object[] { 100, BinaryOutputFormat.NATIVE_BLOCK_SIZE, parallelism });
+			params.add(new Object[] { 100, 1000, parallelism });
+			params.add(new Object[] { 100, 1 << 20, parallelism });
+			params.add(new Object[] { 10000, 1000, parallelism });
+			params.add(new Object[] { 10000, 1 << 20, parallelism });
 		}
 		return params;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java
index 90347b8..e421f4f 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java
@@ -35,8 +35,8 @@ public class SerializedFormatTest extends SequentialFormatTestBase<Record> {
 
 	private BlockInfo info;
 
-	public SerializedFormatTest(int numberOfRecords, long blockSize, int degreeOfParallelism){
-		super(numberOfRecords, blockSize, degreeOfParallelism);
+	public SerializedFormatTest(int numberOfRecords, long blockSize, int parallelism){
+		super(numberOfRecords, blockSize, parallelism);
 	}
 
 	@Before

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-dist/src/main/resources/flink-conf.yaml
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml
index 894137f..2f2eb22 100644
--- a/flink-dist/src/main/resources/flink-conf.yaml
+++ b/flink-dist/src/main/resources/flink-conf.yaml
@@ -31,7 +31,7 @@ taskmanager.heap.mb: 512
 
 taskmanager.numberOfTaskSlots: 1
 
-parallelization.degree.default: 1
+parallelism.default: 1
 
 #==============================================================================
 # Web Frontend

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
index b045100..ed8d1ca 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
@@ -222,7 +222,7 @@ public abstract class DataSet<T> {
 	 * The function is called once per parallel partition of the data,
 	 * and the entire partition is available through the given Iterator.
 	 * The number of elements that each instance of the MapPartition function
-	 * sees is non deterministic and depends on the degree of parallelism of the operation.
+	 * sees is non deterministic and depends on the parallelism of the operation.
 	 *
 	 * This function is intended for operations that cannot transform individual elements,
 	 * requires no grouping of elements. To transform individual elements,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index aac3147..9d1fc36 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -133,14 +133,14 @@ public abstract class ExecutionEnvironment {
 	}
 
 	/**
-	 * Gets the degree of parallelism with which operation are executed by default. Operations can
-	 * individually override this value to use a specific degree of parallelism via
+	 * Gets the parallelism with which operation are executed by default. Operations can
+	 * individually override this value to use a specific parallelism via
 	 * {@link Operator#setParallelism(int)}. Other operations may need to run with a different
-	 * degree of parallelism - for example calling
+	 * parallelism - for example calling
 	 * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire
-	 * set will insert eventually an operation that runs non-parallel (degree of parallelism of one).
+	 * set will insert eventually an operation that runs non-parallel (parallelism of one).
 	 * 
-	 * @return The degree of parallelism used by operations, unless they override that value. This method
+	 * @return The parallelism used by operations, unless they override that value. This method
 	 *         returns {@code -1}, if the environments default parallelism should be used.
 	 * @deprecated Please use {@link #getParallelism}
 	 */
@@ -165,14 +165,14 @@ public abstract class ExecutionEnvironment {
 	}
 	
 	/**
-	 * 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.
 	 * The {@link LocalEnvironment} uses by default a value equal to the number of hardware
 	 * contexts (CPU cores / threads). When executing the program via the command line client 
-	 * from a JAR file, the default degree of parallelism is the one configured for that setup.
+	 * from a JAR file, the default parallelism is the one configured for that setup.
 	 * 
 	 * @param parallelism The parallelism
 	 * @deprecated Please use {@link #setParallelism}
@@ -592,7 +592,7 @@ public abstract class ExecutionEnvironment {
 	 * via {@link #fromCollection(Collection, TypeInformation)}.
 	 * <p>
 	 * Note that this operation will result in a non-parallel data source, i.e. a data source with
-	 * a degree of parallelism of one.
+	 * a parallelism of one.
 	 * 
 	 * @param data The collection of elements to create the data set from.
 	 * @return A DataSet representing the given collection.
@@ -621,7 +621,7 @@ public abstract class ExecutionEnvironment {
 	 * if needed.
 	 * <p>
 	 * Note that this operation will result in a non-parallel data source, i.e. a data source with
-	 * a degree of parallelism of one.
+	 * a parallelism of one.
 	 * <p>
 	 * The returned DataSet is typed to the given TypeInformation.
 	 *  
@@ -651,7 +651,7 @@ public abstract class ExecutionEnvironment {
 	 * framework may move it to a remote environment, if needed.
 	 * <p>
 	 * Note that this operation will result in a non-parallel data source, i.e. a data source with
-	 * a degree of parallelism of one.
+	 * a parallelism of one.
 	 * 
 	 * @param data The collection of elements to create the data set from.
 	 * @param type The class of the data produced by the iterator. Must not be a generic class.
@@ -674,7 +674,7 @@ public abstract class ExecutionEnvironment {
 	 * framework may move it to a remote environment, if needed.
 	 * <p>
 	 * Note that this operation will result in a non-parallel data source, i.e. a data source with
-	 * a degree of parallelism of one.
+	 * a parallelism of one.
 	 * 
 	 * @param data The collection of elements to create the data set from.
 	 * @param type The TypeInformation for the produced data set.
@@ -702,7 +702,7 @@ public abstract class ExecutionEnvironment {
 	 * via {@link #fromCollection(Collection, TypeInformation)}.
 	 * <p>
 	 * Note that this operation will result in a non-parallel data source, i.e. a data source with
-	 * a degree of parallelism of one.
+	 * a parallelism of one.
 	 * 
 	 * @param data The elements to make up the data set.
 	 * @return A DataSet representing the given list of elements.
@@ -924,8 +924,8 @@ public abstract class ExecutionEnvironment {
 		OperatorTranslation translator = new OperatorTranslation();
 		JavaPlan plan = translator.translateToPlan(this.sinks, jobName);
 
-		if (getDegreeOfParallelism() > 0) {
-			plan.setDefaultParallelism(getDegreeOfParallelism());
+		if (getParallelism() > 0) {
+			plan.setDefaultParallelism(getParallelism());
 		}
 		plan.setExecutionConfig(getConfig());
 		// Check plan for GenericTypeInfo's and register the types at the serializers.
@@ -1024,18 +1024,18 @@ public abstract class ExecutionEnvironment {
 	/**
 	 * Creates a {@link CollectionEnvironment} that uses Java Collections underneath. This will execute in a
 	 * single thread in the current JVM. It is very fast but will fail if the data does not fit into
-	 * memory. Degree of parallelism will always be 1. This is useful during implementation and for debugging.
+	 * memory. parallelism will always be 1. This is useful during implementation and for debugging.
 	 * @return A Collection Environment
 	 */
 	public static CollectionEnvironment createCollectionsEnvironment(){
 		CollectionEnvironment ce = new CollectionEnvironment();
-		ce.setDegreeOfParallelism(1);
+		ce.setParallelism(1);
 		return ce;
 	}
 
 	/**
 	 * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a
-	 * multi-threaded fashion in the same JVM as the environment was created in. The default degree of
+	 * multi-threaded fashion in the same JVM as the environment was created in. The default
 	 * parallelism of the local environment is the number of hardware contexts (CPU cores / threads),
 	 * unless it was specified differently by {@link #setDefaultLocalParallelism(int)}.
 	 * 
@@ -1048,22 +1048,22 @@ public abstract class ExecutionEnvironment {
 	/**
 	 * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a
 	 * multi-threaded fashion in the same JVM as the environment was created in. It will use the
-	 * degree of parallelism specified in the parameter.
+	 * parallelism specified in the parameter.
 	 * 
-	 * @param degreeOfParallelism The degree of parallelism for the local environment.
-	 * @return A local execution environment with the specified degree of parallelism.
+	 * @param parallelism The parallelism for the local environment.
+	 * @return A local execution environment with the specified parallelism.
 	 */
-	public static LocalEnvironment createLocalEnvironment(int degreeOfParallelism) {
+	public static LocalEnvironment createLocalEnvironment(int parallelism) {
 		LocalEnvironment lee = new LocalEnvironment();
-		lee.setDegreeOfParallelism(degreeOfParallelism);
+		lee.setParallelism(parallelism);
 		return lee;
 	}
 	
 	/**
 	 * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program 
 	 * to a cluster for execution. Note that all file paths used in the program must be accessible from the
-	 * cluster. The execution will use the cluster's default degree of parallelism, unless the parallelism is
-	 * set explicitly via {@link ExecutionEnvironment#setDegreeOfParallelism(int)}.
+	 * cluster. The execution will use the cluster's default parallelism, unless the parallelism is
+	 * set explicitly via {@link ExecutionEnvironment#setParallelism(int)}.
 	 * 
 	 * @param host The host name or address of the master (JobManager), where the program should be executed.
 	 * @param port The port of the master (JobManager), where the program should be executed. 
@@ -1079,19 +1079,19 @@ public abstract class ExecutionEnvironment {
 	/**
 	 * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program 
 	 * to a cluster for execution. Note that all file paths used in the program must be accessible from the
-	 * cluster. The execution will use the specified degree of parallelism.
+	 * cluster. The execution will use the specified parallelism.
 	 * 
 	 * @param host The host name or address of the master (JobManager), where the program should be executed.
 	 * @param port The port of the master (JobManager), where the program should be executed. 
-	 * @param degreeOfParallelism The degree of parallelism to use during the execution.
+	 * @param parallelism The parallelism to use during the execution.
 	 * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses
 	 *                 user-defined functions, user-defined input formats, or any libraries, those must be
 	 *                 provided in the JAR files.
 	 * @return A remote environment that executes the program on a cluster.
 	 */
-	public static ExecutionEnvironment createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles) {
+	public static ExecutionEnvironment createRemoteEnvironment(String host, int port, int parallelism, String... jarFiles) {
 		RemoteEnvironment rec = new RemoteEnvironment(host, port, jarFiles);
-		rec.setDegreeOfParallelism(degreeOfParallelism);
+		rec.setParallelism(parallelism);
 		return rec;
 	}
 	
@@ -1099,10 +1099,10 @@ public abstract class ExecutionEnvironment {
 	 * Sets the default parallelism that will be used for the local execution environment created by
 	 * {@link #createLocalEnvironment()}.
 	 * 
-	 * @param degreeOfParallelism The degree of parallelism to use as the default local parallelism.
+	 * @param parallelism The parallelism to use as the default local parallelism.
 	 */
-	public static void setDefaultLocalParallelism(int degreeOfParallelism) {
-		defaultLocalDop = degreeOfParallelism;
+	public static void setDefaultLocalParallelism(int parallelism) {
+		defaultLocalDop = parallelism;
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
index e7daf11..beecaf6 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
@@ -25,10 +25,10 @@ import org.apache.flink.api.common.PlanExecutor;
 
 /**
  * An {@link ExecutionEnvironment} that runs the program locally, multi-threaded, in the JVM where the
- * environment is instantiated. When this environment is instantiated, it uses a default degree of parallelism
+ * environment is instantiated. When this environment is instantiated, it uses a default parallelism
  * of {@code 1}. Local environments can also be instantiated through
  * {@link ExecutionEnvironment#createLocalEnvironment()} and {@link ExecutionEnvironment#createLocalEnvironment(int)}.
- * The former version will pick a default degree of parallelism equal to the number of hardware contexts in the local
+ * The former version will pick a default parallelism equal to the number of hardware contexts in the local
  * machine.
  */
 public class LocalEnvironment extends ExecutionEnvironment {
@@ -62,7 +62,7 @@ public class LocalEnvironment extends ExecutionEnvironment {
 	
 	@Override
 	public String toString() {
-		return "Local Environment (DOP = " + (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism())
+		return "Local Environment (parallelism = " + (getParallelism() == -1 ? "default" : getParallelism())
 				+ ") : " + getIdString();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
index c0695e5..df18bbf 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.PlanExecutor;
 /**
  * An {@link ExecutionEnvironment} that sends programs 
  * to a cluster for execution. Note that all file paths used in the program must be accessible from the
- * cluster. The execution will use the cluster's default degree of parallelism, unless the parallelism is
- * set explicitly via {@link ExecutionEnvironment#setDegreeOfParallelism(int)}.
+ * cluster. The execution will use the cluster's default parallelism, unless the parallelism is
+ * set explicitly via {@link ExecutionEnvironment#setParallelism(int)}.
  */
 public class RemoteEnvironment extends ExecutionEnvironment {
 	
@@ -72,7 +72,7 @@ public class RemoteEnvironment extends ExecutionEnvironment {
 	@Override
 	public String getExecutionPlan() throws Exception {
 		Plan p = createProgramPlan("unnamed", false);
-		p.setDefaultParallelism(getDegreeOfParallelism());
+		p.setDefaultParallelism(getParallelism());
 		registerCachedFilesWithPlan(p);
 		
 		PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, jarFiles);
@@ -81,7 +81,7 @@ public class RemoteEnvironment extends ExecutionEnvironment {
 
 	@Override
 	public String toString() {
-		return "Remote Environment (" + this.host + ":" + this.port + " - DOP = " + 
-				(getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + ") : " + getIdString();
+		return "Remote Environment (" + this.host + ":" + this.port + " - parallelism = " +
+				(getParallelism() == -1 ? "default" : getParallelism()) + ") : " + getIdString();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java
index 7c09518..c8f8bbc 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/SortPartitionOperator.java
@@ -169,10 +169,10 @@ public class SortPartitionOperator<T> extends SingleInputOperator<T, T, SortPart
 		noop.setInput(input);
 		if(this.getParallelism() < 0) {
 			// use parallelism of input if not explicitly specified
-			noop.setDegreeOfParallelism(input.getDegreeOfParallelism());
+			noop.setParallelism(input.getParallelism());
 		} else {
 			// use explicitly specified parallelism
-			noop.setDegreeOfParallelism(this.getParallelism());
+			noop.setParallelism(this.getParallelism());
 		}
 
 		return noop;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java
index 41410da..8e028eb 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/io/PrintingOutputFormat.java
@@ -73,7 +73,7 @@ public class PrintingOutputFormat<T> implements OutputFormat<T> {
 		// get the target stream
 		this.stream = this.target == STD_OUT ? System.out : System.err;
 		
-		// set the prefix if we have a >1 DOP
+		// set the prefix if we have a >1 parallelism
 		this.prefix = (numTasks > 1) ? ((taskNumber+1) + "> ") : null;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
index 66821ae..daaedac 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java
@@ -193,8 +193,8 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 			
 			// set input
 			po.setInput(input);
-			// set dop
-			po.setDegreeOfParallelism(this.getParallelism());
+			// set parallelism
+			po.setParallelism(this.getParallelism());
 			
 			return po;
 		}
@@ -209,7 +209,7 @@ public class AggregateOperator<IN> extends SingleInputOperator<IN, IN, Aggregate
 			po.setCombinable(true);
 			
 			po.setInput(input);
-			po.setDegreeOfParallelism(this.getParallelism());
+			po.setParallelism(this.getParallelism());
 			po.setCustomPartitioner(grouping.getCustomPartitioner());
 			
 			SingleInputSemanticProperties props = new SingleInputSemanticProperties();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
index b33139b..a051eb0 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
@@ -191,7 +191,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			po = translateSelectorFunctionCoGroup(selectorKeys1, selectorKeys2, function,
 					getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
 
-			po.setDegreeOfParallelism(getParallelism());
+			po.setParallelism(getParallelism());
 			po.setCustomPartitioner(customPartitioner);
 		}
 		else if (keys2 instanceof Keys.SelectorFunctionKeys) {
@@ -204,7 +204,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			po = translateSelectorFunctionCoGroupRight(logicalKeyPositions1, selectorKeys2, function,
 							getInput1Type(), getInput2Type(), getResultType(), name, input1, input2);
 
-			po.setDegreeOfParallelism(getParallelism());
+			po.setParallelism(getParallelism());
 			po.setCustomPartitioner(customPartitioner);
 		}
 		else if (keys1 instanceof Keys.SelectorFunctionKeys) {
@@ -242,7 +242,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		}
 		
 		// configure shared characteristics
-		po.setDegreeOfParallelism(getParallelism());
+		po.setParallelism(getParallelism());
 		po.setCustomPartitioner(customPartitioner);
 		
 		if (groupSortKeyOrderFirst.size() > 0) {
@@ -292,9 +292,9 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 
 		keyMapper1.setInput(input1);
 		keyMapper2.setInput(input2);
-		// set dop
-		keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
-		keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
+		// set parallelism
+		keyMapper1.setParallelism(input1.getParallelism());
+		keyMapper2.setParallelism(input2.getParallelism());
 
 		return cogroup;
 	}
@@ -338,8 +338,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		cogroup.setSecondInput(keyMapper2);
 
 		keyMapper2.setInput(input2);
-		// set dop
-		keyMapper2.setDegreeOfParallelism(input2.getDegreeOfParallelism());
+		// set parallelism
+		keyMapper2.setParallelism(input2.getParallelism());
 
 		return cogroup;
 	}
@@ -382,8 +382,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		cogroup.setSecondInput(input2);
 
 		keyMapper1.setInput(input1);
-		// set dop
-		keyMapper1.setDegreeOfParallelism(input1.getDegreeOfParallelism());
+		// set parallelism
+		keyMapper1.setParallelism(input1.getParallelism());
 
 		return cogroup;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
index acfb47a..5ed3e40 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
@@ -95,7 +95,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 
 		po.setFirstInput(input1);
 		po.setSecondInput(input2);
-		po.setDegreeOfParallelism(getParallelism());
+		po.setParallelism(getParallelism());
 		po.setCrossHint(hint);
 
 		return po;

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
index e646891..83ec021 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
@@ -45,7 +45,7 @@ public class DataSink<T> {
 	
 	private String name;
 	
-	private int dop = -1;
+	private int parallelism = -1;
 
 	private Configuration parameters;
 
@@ -234,13 +234,13 @@ public class DataSink<T> {
 		if(this.parameters != null) {
 			sink.getParameters().addAll(this.parameters);
 		}
-		// set dop
-		if(this.dop > 0) {
-			// use specified dop
-			sink.setDegreeOfParallelism(this.dop);
+		// set parallelism
+		if(this.parallelism > 0) {
+			// use specified parallelism
+			sink.setParallelism(this.parallelism);
 		} else {
-			// if no dop has been specified, use dop of input operator to enable chaining
-			sink.setDegreeOfParallelism(input.getDegreeOfParallelism());
+			// if no parallelism has been specified, use parallelism of input operator to enable chaining
+			sink.setParallelism(input.getParallelism());
 		}
 
 		if(this.sortKeyPositions != null) {
@@ -263,27 +263,27 @@ public class DataSink<T> {
 	}
 	
 	/**
-	 * Returns the degree of parallelism of this data sink.
+	 * Returns the parallelism of this data sink.
 	 * 
-	 * @return The degree of parallelism of this data sink.
+	 * @return The parallelism of this data sink.
 	 */
 	public int getParallelism() {
-		return this.dop;
+		return this.parallelism;
 	}
 	
 	/**
-	 * Sets the degree of parallelism for this data sink.
+	 * Sets the parallelism for this data sink.
 	 * The degree must be 1 or more.
 	 * 
-	 * @param dop The degree of parallelism for this data sink.
-	 * @return This data sink with set degree of parallelism.
+	 * @param parallelism The parallelism for this data sink.
+	 * @return This data sink with set parallelism.
 	 */
-	public DataSink<T> setParallelism(int dop) {
+	public DataSink<T> setParallelism(int parallelism) {
 		
-		if(dop < 1) {
+		if(parallelism < 1) {
 			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
 		}
-		this.dop = dop;
+		this.parallelism = parallelism;
 		
 		return this;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java
index d6e511a..728985a 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSource.java
@@ -65,7 +65,7 @@ public class DataSource<OUT> extends Operator<OUT, DataSource<OUT>> {
 		this.inputFormat = inputFormat;
 		
 		if (inputFormat instanceof NonParallelInput) {
-			this.dop = 1;
+			this.parallelism = 1;
 		}
 	}
 
@@ -126,7 +126,7 @@ public class DataSource<OUT> extends Operator<OUT, DataSource<OUT>> {
 		@SuppressWarnings({ "unchecked", "rawtypes" })
 		GenericDataSourceBase<OUT, ?> source = new GenericDataSourceBase(this.inputFormat,
 				new OperatorInformation<OUT>(getType()), name);
-		source.setDegreeOfParallelism(dop);
+		source.setParallelism(parallelism);
 		if(this.parameters != null) {
 			source.getParameters().addAll(this.parameters);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
index aec4950..4d61aec 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java
@@ -165,19 +165,19 @@ public class DeltaIteration<ST, WT> {
 	}
 	
 	/**
-	 * Sets the degree of parallelism for the iteration.
+	 * Sets the parallelism for the iteration.
 	 *
-	 * @param parallelism The degree of parallelism.
+	 * @param parallelism The parallelism.
 	 * @return The iteration object, for function call chaining.
 	 */
 	public DeltaIteration<ST, WT> parallelism(int parallelism) {
-		Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default).");
+		Validate.isTrue(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default).");
 		this.parallelism = parallelism;
 		return this;
 	}
 	
 	/**
-	 * Gets the iteration's degree of parallelism.
+	 * Gets the iteration's parallelism.
 	 * 
 	 * @return The iterations parallelism, or -1, if not set.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
index d570fc2..686823c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java
@@ -88,7 +88,7 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 
 			po.setCombinable(true);
 			po.setInput(input);
-			po.setDegreeOfParallelism(getParallelism());
+			po.setParallelism(getParallelism());
 			
 			// make sure that distinct preserves the partitioning for the fields on which they operate
 			if (getType().isTupleType()) {
@@ -113,7 +113,7 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 			PlanUnwrappingReduceGroupOperator<T, T, ?> po = translateSelectorFunctionDistinct(
 							selectorKeys, function, getInputType(), getResultType(), name, input);
 			
-			po.setDegreeOfParallelism(this.getParallelism());
+			po.setParallelism(this.getParallelism());
 			
 			return po;
 		}
@@ -145,7 +145,7 @@ public class DistinctOperator<T> extends SingleInputOperator<T, T, DistinctOpera
 		mapper.setInput(input);
 		
 		// set the mapper's parallelism to the input parallelism to make sure it is chained
-		mapper.setDegreeOfParallelism(input.getDegreeOfParallelism());
+		mapper.setParallelism(input.getParallelism());
 		
 		return reducer;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
index 56bea50..f55de1c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/FilterOperator.java
@@ -57,13 +57,13 @@ public class FilterOperator<T> extends SingleInputUdfOperator<T, T, FilterOperat
 		PlanFilterOperator<T> po = new PlanFilterOperator<T>(function, name, getInputType());
 		po.setInput(input);
 		
-		// set dop
+		// set parallelism
 		if (getParallelism() > 0) {
-			// use specified dop
-			po.setDegreeOfParallelism(getParallelism());
+			// use specified parallelism
+			po.setParallelism(getParallelism());
 		} else {
-			// if no dop has been specified, use dop of input operator to enable chaining
-			po.setDegreeOfParallelism(input.getDegreeOfParallelism());
+			// if no parallelism has been specified, use parallelism of input operator to enable chaining
+			po.setParallelism(input.getParallelism());
 		}
 		
 		return po;


[2/9] flink git commit: [FLINK-1679] use a consistent name for parallelism

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java
index 02a6e38..c2ec55a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java
@@ -111,17 +111,17 @@ public class JoinITCase extends RecordAPITestBase {
 				new ContractITCaseInputFormat(), leftInPath);
 		DelimitedInputFormat.configureDelimitedFormat(input_left)
 			.recordDelimiter('\n');
-		input_left.setDegreeOfParallelism(config.getInteger("MatchTest#NoSubtasks", 1));
+		input_left.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1));
 
 		FileDataSource input_right = new FileDataSource(
 				new ContractITCaseInputFormat(), rightInPath);
 		DelimitedInputFormat.configureDelimitedFormat(input_right)
 			.recordDelimiter('\n');
-		input_right.setDegreeOfParallelism(config.getInteger("MatchTest#NoSubtasks", 1));
+		input_right.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1));
 
 		JoinOperator testMatcher = JoinOperator.builder(new TestMatcher(), StringValue.class, 0, 0)
 			.build();
-		testMatcher.setDegreeOfParallelism(config.getInteger("MatchTest#NoSubtasks", 1));
+		testMatcher.setParallelism(config.getInteger("MatchTest#NoSubtasks", 1));
 		testMatcher.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY,
 				config.getString("MatchTest#LocalStrategy", ""));
 		if (config.getString("MatchTest#ShipStrategy", "").equals("BROADCAST_FIRST")) {
@@ -141,7 +141,7 @@ public class JoinITCase extends RecordAPITestBase {
 
 		FileDataSink output = new FileDataSink(
 				new ContractITCaseOutputFormat(), resultPath);
-		output.setDegreeOfParallelism(1);
+		output.setParallelism(1);
 
 		output.setInput(testMatcher);
 		testMatcher.setFirstInput(input_left);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java
index 0d7bfdf..28b9501 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java
@@ -100,14 +100,14 @@ public class MapITCase extends RecordAPITestBase {
 				new ContractITCaseInputFormat(), inPath);
 		DelimitedInputFormat.configureDelimitedFormat(input)
 			.recordDelimiter('\n');
-		input.setDegreeOfParallelism(config.getInteger("MapTest#NoSubtasks", 1));
+		input.setParallelism(config.getInteger("MapTest#NoSubtasks", 1));
 
 		MapOperator testMapper = MapOperator.builder(new TestMapper()).build();
-		testMapper.setDegreeOfParallelism(config.getInteger("MapTest#NoSubtasks", 1));
+		testMapper.setParallelism(config.getInteger("MapTest#NoSubtasks", 1));
 
 		FileDataSink output = new FileDataSink(
 				new ContractITCaseOutputFormat(), resultPath);
-		output.setDegreeOfParallelism(1);
+		output.setParallelism(1);
 
 		output.setInput(testMapper);
 		testMapper.setInput(input);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java
index 3c8d372..498a638 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java
@@ -114,11 +114,11 @@ public class ReduceITCase extends RecordAPITestBase {
 				new ContractITCaseInputFormat(), inPath);
 		DelimitedInputFormat.configureDelimitedFormat(input)
 			.recordDelimiter('\n');
-		input.setDegreeOfParallelism(config.getInteger("ReduceTest#NoSubtasks", 1));
+		input.setParallelism(config.getInteger("ReduceTest#NoSubtasks", 1));
 
 		ReduceOperator testReducer = ReduceOperator.builder(new TestReducer(), StringValue.class, 0)
 			.build();
-		testReducer.setDegreeOfParallelism(config.getInteger("ReduceTest#NoSubtasks", 1));
+		testReducer.setParallelism(config.getInteger("ReduceTest#NoSubtasks", 1));
 		testReducer.getParameters().setString(Optimizer.HINT_LOCAL_STRATEGY,
 				config.getString("ReduceTest#LocalStrategy", ""));
 		testReducer.getParameters().setString(Optimizer.HINT_SHIP_STRATEGY,
@@ -126,7 +126,7 @@ public class ReduceITCase extends RecordAPITestBase {
 
 		FileDataSink output = new FileDataSink(
 				new ContractITCaseOutputFormat(), resultPath);
-		output.setDegreeOfParallelism(1);
+		output.setParallelism(1);
 
 		output.setInput(testReducer);
 		testReducer.setInput(input);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java
index 481ae2d..b833421 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java
@@ -111,20 +111,20 @@ public class UnionITCase extends RecordAPITestBase {
 			new ContractITCaseInputFormat(), input1Path);
 		DelimitedInputFormat.configureDelimitedFormat(input1)
 			.recordDelimiter('\n');
-		input1.setDegreeOfParallelism(config.getInteger("UnionTest#NoSubtasks", 1));
+		input1.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1));
 		
 		FileDataSource input2 = new FileDataSource(
 				new ContractITCaseInputFormat(), input2Path);
 		DelimitedInputFormat.configureDelimitedFormat(input2)
 			.recordDelimiter('\n');
-		input2.setDegreeOfParallelism(config.getInteger("UnionTest#NoSubtasks", 1));
+		input2.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1));
 		
 		MapOperator testMapper = MapOperator.builder(new TestMapper()).build();
-		testMapper.setDegreeOfParallelism(config.getInteger("UnionTest#NoSubtasks", 1));
+		testMapper.setParallelism(config.getInteger("UnionTest#NoSubtasks", 1));
 
 		FileDataSink output = new FileDataSink(
 				new ContractITCaseOutputFormat(), resultPath);
-		output.setDegreeOfParallelism(1);
+		output.setParallelism(1);
 
 		output.setInput(testMapper);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java
index 944de98..c214dbd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionSinkITCase.java
@@ -50,7 +50,7 @@ public class UnionSinkITCase extends RecordAPITestBase {
 	
 	public UnionSinkITCase(Configuration testConfig) {
 		super(testConfig);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	private static final String MAP_IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" +
@@ -122,7 +122,7 @@ public class UnionSinkITCase extends RecordAPITestBase {
 		output.addInput(testMapper2);
 		
 		Plan plan = new Plan(output);
-		plan.setDefaultParallelism(DOP);
+		plan.setDefaultParallelism(parallelism);
 
 		Optimizer pc = new Optimizer(new DataStatistics());
 		OptimizedPlan op = pc.compile(plan);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java
index 16d2517..86eafe5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/CollectionSourceTest.java
@@ -42,12 +42,12 @@ import org.apache.flink.util.Collector;
 @SuppressWarnings("deprecation")
 public class CollectionSourceTest extends RecordAPITestBase {
 
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
 	protected String resultPath;
 
 	public CollectionSourceTest(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	public static class Join extends JoinFunction {
@@ -122,7 +122,7 @@ public class CollectionSourceTest extends RecordAPITestBase {
 
 	@Override
 	protected Plan getTestJob() {
-		return getPlan(DOP, resultPath);
+		return getPlan(parallelism, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java
index 0a6d29d..613de78 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/ComputeEdgeDegreesITCase.java
@@ -40,7 +40,7 @@ public class ComputeEdgeDegreesITCase extends RecordAPITestBase {
 	
 	public ComputeEdgeDegreesITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -64,7 +64,7 @@ public class ComputeEdgeDegreesITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("NumSubtasks", DOP);
+		config.setInteger("NumSubtasks", parallelism);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
index c758f32..6b0f01c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesOnEdgesWithDegreesITCase.java
@@ -39,7 +39,7 @@ public class EnumTrianglesOnEdgesWithDegreesITCase extends RecordAPITestBase {
 	
 	public EnumTrianglesOnEdgesWithDegreesITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -65,7 +65,7 @@ public class EnumTrianglesOnEdgesWithDegreesITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("NumSubtasks", DOP);
+		config.setInteger("NumSubtasks", parallelism);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java
index f0a1288..b96a31c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/EnumTrianglesRDFITCase.java
@@ -59,7 +59,7 @@ public class EnumTrianglesRDFITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		EnumTrianglesRdfFoaf enumTriangles = new EnumTrianglesRdfFoaf();
 		return enumTriangles.getPlan(
-				String.valueOf(config.getInteger("NumSubtasks", DOP)), edgesPath, resultPath);
+				String.valueOf(config.getInteger("NumSubtasks", parallelism)), edgesPath, resultPath);
 	}
 
 	@Override
@@ -70,7 +70,7 @@ public class EnumTrianglesRDFITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("NumSubtasks", DOP);
+		config.setInteger("NumSubtasks", parallelism);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java
index f23b129..15486cb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingITCase.java
@@ -45,7 +45,7 @@ public class GlobalSortingITCase extends RecordAPITestBase {
 	private String sortedRecords;
 
 	public GlobalSortingITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -86,7 +86,7 @@ public class GlobalSortingITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		GlobalSort globalSort = new GlobalSort();
-		return globalSort.getPlan(Integer.valueOf(DOP).toString(), recordsPath, resultPath);
+		return globalSort.getPlan(Integer.valueOf(parallelism).toString(), recordsPath, resultPath);
 	}
 
 	@Override
@@ -108,7 +108,7 @@ public class GlobalSortingITCase extends RecordAPITestBase {
 			String output        = (args.length > 2 ? args[2] : "");
 			
 			FileDataSource source = new FileDataSource(CsvInputFormat.class, recordsPath);
-			source.setDegreeOfParallelism(numSubtasks);
+			source.setParallelism(numSubtasks);
 			CsvInputFormat.configureRecordFormat(source)
 				.recordDelimiter('\n')
 				.fieldDelimiter('|')
@@ -116,7 +116,7 @@ public class GlobalSortingITCase extends RecordAPITestBase {
 			
 			FileDataSink sink =
 				new FileDataSink(CsvOutputFormat.class, output);
-			sink.setDegreeOfParallelism(numSubtasks);
+			sink.setParallelism(numSubtasks);
 			CsvOutputFormat.configureRecordFormat(sink)
 				.recordDelimiter('\n')
 				.fieldDelimiter('|')

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java
index b6b8b9d..e79b807 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GlobalSortingMixedOrderITCase.java
@@ -53,7 +53,7 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 	private String sortedRecords;
 
 	public GlobalSortingMixedOrderITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -91,7 +91,7 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		GlobalSort globalSort = new GlobalSort();
-		return globalSort.getPlan(Integer.valueOf(DOP).toString(), recordsPath, resultPath);
+		return globalSort.getPlan(Integer.valueOf(parallelism).toString(), recordsPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java
index 368f9af..c1ad83d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/GroupOrderReduceITCase.java
@@ -65,7 +65,7 @@ public class GroupOrderReduceITCase extends RecordAPITestBase {
 	
 	public GroupOrderReduceITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	
@@ -78,7 +78,7 @@ public class GroupOrderReduceITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		
-		int dop = this.config.getInteger("GroupOrderTest#NumSubtasks", 1);
+		int parallelism = this.config.getInteger("GroupOrderTest#NumSubtasks", 1);
 		
 		@SuppressWarnings("unchecked")
 		CsvInputFormat format = new CsvInputFormat(',', IntValue.class, IntValue.class);
@@ -99,7 +99,7 @@ public class GroupOrderReduceITCase extends RecordAPITestBase {
 			.field(IntValue.class, 1);
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(dop);
+		p.setDefaultParallelism(parallelism);
 		return p;
 	}
 
@@ -110,7 +110,7 @@ public class GroupOrderReduceITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("GroupOrderTest#NumSubtasks", DOP);
+		config.setInteger("GroupOrderTest#NumSubtasks", parallelism);
 		return toParameterList(config);
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java
index 231f196..1a0cae2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/PairwiseSPITCase.java
@@ -71,7 +71,7 @@ public class PairwiseSPITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		PairwiseSP a2aSP = new PairwiseSP();
 		return a2aSP.getPlan(
-				String.valueOf(config.getInteger("All2AllSPTest#NoSubtasks", DOP)),
+				String.valueOf(config.getInteger("All2AllSPTest#NoSubtasks", parallelism)),
 				rdfDataPath,
 				resultPath,
 				"true");
@@ -85,7 +85,7 @@ public class PairwiseSPITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("All2AllSPTest#NoSubtasks", DOP);
+		config.setInteger("All2AllSPTest#NoSubtasks", parallelism);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java
index b8eb4d2..349275c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery10ITCase.java
@@ -205,7 +205,7 @@ public class TPCHQuery10ITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("TPCHQuery10Test#NoSubtasks", DOP);
+		config.setInteger("TPCHQuery10Test#NoSubtasks", parallelism);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java
index 461d6c0..a0236c2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3ITCase.java
@@ -126,7 +126,7 @@ public class TPCHQuery3ITCase extends RecordAPITestBase {
 
 	public TPCHQuery3ITCase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -141,7 +141,7 @@ public class TPCHQuery3ITCase extends RecordAPITestBase {
 
 		TPCHQuery3 tpch3 = new TPCHQuery3();
 		return tpch3.getPlan(
-				String.valueOf(config.getInteger("dop", 1)), 
+				String.valueOf(config.getInteger("parallelism", 1)),
 				ordersPath,
 				lineitemsPath,
 				resultPath);
@@ -155,7 +155,7 @@ public class TPCHQuery3ITCase extends RecordAPITestBase {
 	@Parameters
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config = new Configuration();
-		config.setInteger("dop", DOP);
+		config.setInteger("parallelism", parallelism);
 		return toParameterList(config);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java
index 84a3930..3ade964 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery3WithUnionITCase.java
@@ -128,7 +128,7 @@ public class TPCHQuery3WithUnionITCase extends RecordAPITestBase {
 	private static final String EXPECTED_RESULT = "5|0|147828.97\n" + "66|0|99188.09\n";
 
 	public TPCHQuery3WithUnionITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 
@@ -146,7 +146,7 @@ public class TPCHQuery3WithUnionITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		TPCHQuery3Unioned tpch3 = new TPCHQuery3Unioned();
 		return tpch3.getPlan(
-				Integer.valueOf(DOP).toString(),
+				Integer.valueOf(parallelism).toString(),
 				orders1Path,
 				orders2Path,
 				partJoin1Path,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java
index 8ddfa30..30c1b3e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery4ITCase.java
@@ -119,7 +119,7 @@ public class TPCHQuery4ITCase extends RecordAPITestBase {
 	private static final String EXPECTED_RESULT = "1-URGENT|2|\n" + "3-MEDIUM|2|\n" + "4-NOT SPECIFIED|4|";
 
 	public TPCHQuery4ITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -132,7 +132,7 @@ public class TPCHQuery4ITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		TPCHQuery4 tpch4 = new TPCHQuery4();
-		return tpch4.getPlan(Integer.valueOf(DOP).toString(), ordersPath, lineitemsPath, resultPath);
+		return tpch4.getPlan(Integer.valueOf(parallelism).toString(), ordersPath, lineitemsPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java
index c329ce3..f092400 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQuery9ITCase.java
@@ -343,7 +343,7 @@ public class TPCHQuery9ITCase extends RecordAPITestBase {
 		+ "IRAN|1996|9672.556\n";
 
 	public TPCHQuery9ITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 		
 
@@ -362,7 +362,7 @@ public class TPCHQuery9ITCase extends RecordAPITestBase {
 	protected Plan getTestJob() {
 		TPCHQuery9 tpch9 = new TPCHQuery9();
 		return tpch9.getPlan(
-				Integer.valueOf(DOP).toString(),
+				Integer.valueOf(parallelism).toString(),
 				partInputPath,
 				partSuppInputPath,
 				ordersInputPath,

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java
index 769120b..2c53ee2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TPCHQueryAsterixITCase.java
@@ -69,7 +69,7 @@ public class TPCHQueryAsterixITCase extends RecordAPITestBase {
 		"2|FURNITURE\n";
 
 	public TPCHQueryAsterixITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 
@@ -83,7 +83,7 @@ public class TPCHQueryAsterixITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		TPCHQueryAsterix tpchBench = new TPCHQueryAsterix();
-		return tpchBench.getPlan(Integer.valueOf(DOP).toString(), ordersPath, custPath, resultPath);
+		return tpchBench.getPlan(Integer.valueOf(parallelism).toString(), ordersPath, custPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java
index 2cb0f8f..a45db2f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/TeraSortITCase.java
@@ -36,7 +36,7 @@ public class TeraSortITCase extends RecordAPITestBase {
 	private String resultPath;
 
 	public TeraSortITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -49,7 +49,7 @@ public class TeraSortITCase extends RecordAPITestBase {
 		String testDataPath = getClass().getResource(INPUT_DATA_FILE).toString();
 		
 		TeraSort ts = new TeraSort();
-		return ts.getPlan(Integer.valueOf(DOP).toString(), testDataPath, resultPath);
+		return ts.getPlan(Integer.valueOf(parallelism).toString(), testDataPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java
index 109e62d..f69567b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WebLogAnalysisITCase.java
@@ -155,7 +155,7 @@ public class WebLogAnalysisITCase extends RecordAPITestBase {
 	private static final String expected = "87|url_24|39\n" + "59|url_28|41\n";
 
 	public WebLogAnalysisITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	@Override
@@ -169,7 +169,7 @@ public class WebLogAnalysisITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WebLogAnalysis relOLAP = new WebLogAnalysis();
-		return relOLAP.getPlan(Integer.valueOf(DOP).toString(), docsPath, ranksPath, visitsPath, resultPath);
+		return relOLAP.getPlan(Integer.valueOf(parallelism).toString(), docsPath, ranksPath, visitsPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java
index 165c2d3..19f3dec 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountITCase.java
@@ -30,7 +30,7 @@ public class WordCountITCase extends RecordAPITestBase {
 	protected String resultPath;
 
 	public WordCountITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	
@@ -43,7 +43,7 @@ public class WordCountITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WordCount wc = new WordCount();
-		return wc.getPlan(Integer.valueOf(DOP).toString(), textPath, resultPath);
+		return wc.getPlan(Integer.valueOf(parallelism).toString(), textPath, resultPath);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java
index 52c815b..4518568 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobTests/WordCountUnionReduceITCase.java
@@ -51,7 +51,7 @@ public class WordCountUnionReduceITCase extends RecordAPITestBase {
 	private String outputPath;
 
 	public WordCountUnionReduceITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 
 
@@ -69,7 +69,7 @@ public class WordCountUnionReduceITCase extends RecordAPITestBase {
 	@Override
 	protected Plan getTestJob() {
 		WordCountUnionReduce wc = new WordCountUnionReduce();
-		return wc.getPlan(this.inputPath, this.outputPath, DOP);
+		return wc.getPlan(this.inputPath, this.outputPath, parallelism);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java
index c87dd64..80ba91a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java
@@ -44,7 +44,7 @@ public class DanglingPageRank implements Program, ProgramDescription {
 	public static final String NUM_VERTICES_CONFIG_PARAM = "pageRank.numVertices";
 		
 	public Plan getPlan(String ... args) {
-		int dop = 1;
+		int parallelism = 1;
 		String pageWithRankInputPath = "";
 		String adjacencyListInputPath = "";
 		String outputPath = "";
@@ -53,7 +53,7 @@ public class DanglingPageRank implements Program, ProgramDescription {
 		long numDanglingVertices = 1;
 
 		if (args.length >= 7) {
-			dop = Integer.parseInt(args[0]);
+			parallelism = Integer.parseInt(args[0]);
 			pageWithRankInputPath = args[1];
 			adjacencyListInputPath = args[2];
 			outputPath = args[3];
@@ -94,12 +94,12 @@ public class DanglingPageRank implements Program, ProgramDescription {
 		FileDataSink out = new FileDataSink(new PageWithRankOutFormat(), outputPath, iteration, "Final Ranks");
 
 		Plan p = new Plan(out, "Dangling PageRank");
-		p.setDefaultParallelism(dop);
+		p.setDefaultParallelism(parallelism);
 		return p;
 	}
 
 	@Override
 	public String getDescription() {
-		return "Parameters: <degree-of-parallelism> <pages-input-path> <edges-input-path> <output-path> <max-iterations> <num-vertices> <num-dangling-vertices>";
+		return "Parameters: <parallelism> <pages-input-path> <edges-input-path> <output-path> <max-iterations> <num-vertices> <num-dangling-vertices>";
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java
index 55e2f57..34d4b60 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/PairwiseSP.java
@@ -399,24 +399,24 @@ public class PairwiseSP implements Program, ProgramDescription {
 		} else {
 			pathsInput = new FileDataSource(new PathInFormat(), paths, "Paths");
 		}
-		pathsInput.setDegreeOfParallelism(numSubTasks);
+		pathsInput.setParallelism(numSubTasks);
 
 		JoinOperator concatPaths = 
 				JoinOperator.builder(new ConcatPaths(), StringValue.class, 0, 1)
 			.name("Concat Paths")
 			.build();
 
-		concatPaths.setDegreeOfParallelism(numSubTasks);
+		concatPaths.setParallelism(numSubTasks);
 
 		CoGroupOperator findShortestPaths = 
 				CoGroupOperator.builder(new FindShortestPath(), StringValue.class, 0, 0)
 			.keyField(StringValue.class, 1, 1)
 			.name("Find Shortest Paths")
 			.build();
-		findShortestPaths.setDegreeOfParallelism(numSubTasks);
+		findShortestPaths.setParallelism(numSubTasks);
 
 		FileDataSink result = new FileDataSink(new PathOutFormat(),output, "New Paths");
-		result.setDegreeOfParallelism(numSubTasks);
+		result.setParallelism(numSubTasks);
 
 		result.setInput(findShortestPaths);
 		findShortestPaths.setFirstInput(pathsInput);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java
index 3abf743..31d992f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java
@@ -132,7 +132,7 @@ public class SimplePageRank implements Program, ProgramDescription {
 	// --------------------------------------------------------------------------------------------
 	
 	public Plan getPlan(String ... args) {
-		int dop = 1;
+		int parallelism = 1;
 		String pageWithRankInputPath = "";
 		String adjacencyListInputPath = "";
 		String outputPath = "";
@@ -140,7 +140,7 @@ public class SimplePageRank implements Program, ProgramDescription {
 		long numVertices = 5;
 
 		if (args.length >= 6) {
-			dop = Integer.parseInt(args[0]);
+			parallelism = Integer.parseInt(args[0]);
 			pageWithRankInputPath = args[1];
 			adjacencyListInputPath = args[2];
 			outputPath = args[3];
@@ -183,12 +183,12 @@ public class SimplePageRank implements Program, ProgramDescription {
 		FileDataSink out = new FileDataSink(new PageWithRankOutFormat(), outputPath, iteration, "Final Ranks");
 
 		Plan p = new Plan(out, "Simple PageRank");
-		p.setDefaultParallelism(dop);
+		p.setDefaultParallelism(parallelism);
 		return p;
 	}
 
 	@Override
 	public String getDescription() {
-		return "Parameters: <degree-of-parallelism> <pages-input-path> <edges-input-path> <output-path> <max-iterations> <num-vertices> <num-dangling-vertices>";
+		return "Parameters: <parallelism> <pages-input-path> <edges-input-path> <output-path> <max-iterations> <num-vertices> <num-dangling-vertices>";
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
index e5f519d..d528f5d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
@@ -54,7 +54,7 @@ public class KMeansBroadcast implements Program, ProgramDescription {
 	@Override
 	public Plan getPlan(String... args) {
 		// parse job parameters
-		int degreeOfParallelism = (args.length > 0 ? Integer.parseInt(args[0]) : 1);
+		int parallelism = (args.length > 0 ? Integer.parseInt(args[0]) : 1);
 		String dataPointInput = (args.length > 1 ? args[1] : "");
 		String clusterInput = (args.length > 2 ? args[2] : "");
 		String output = (args.length > 3 ? args[3] : "");
@@ -99,7 +99,7 @@ public class KMeansBroadcast implements Program, ProgramDescription {
 		FileDataSink newClusterPoints = new FileDataSink(new PointOutFormat(), output, iter, "New Center Positions");
 
 		Plan plan = new Plan(newClusterPoints, "K-Means");
-		plan.setDefaultParallelism(degreeOfParallelism);
+		plan.setDefaultParallelism(parallelism);
 		return plan;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java
index 4069f9a..8d75d47 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansCross.java
@@ -53,7 +53,7 @@ public class KMeansCross implements Program, ProgramDescription {
 
 		// create DataSourceContract for cluster center input
 		FileDataSource initialClusterPoints = new FileDataSource(new PointInFormat(), clusterInput, "Centers");
-		initialClusterPoints.setDegreeOfParallelism(1);
+		initialClusterPoints.setParallelism(1);
 		
 		BulkIteration iteration = new BulkIteration("K-Means Loop");
 		iteration.setInput(initialClusterPoints);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java
index 74b8f4c..b948804 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/MergeOnlyJoin.java
@@ -89,13 +89,13 @@ public class MergeOnlyJoin implements Program {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat format2 = new CsvInputFormat('|', IntValue.class, IntValue.class);
 		FileDataSource input2 = new FileDataSource(format2, input2Path, "Input 2");
-		input2.setDegreeOfParallelism(numSubtasksInput2);
+		input2.setParallelism(numSubtasksInput2);
 
 		ReduceOperator aggInput2 = ReduceOperator.builder(DummyReduce.class, IntValue.class, 0)
 			.input(input2)
 			.name("AggLines")
 			.build();
-		aggInput2.setDegreeOfParallelism(numSubtasksInput2);
+		aggInput2.setParallelism(numSubtasksInput2);
 		
 		// create JoinOperator for joining Orders and LineItems
 		JoinOperator joinLiO = JoinOperator.builder(JoinInputs.class, IntValue.class, 0, 0)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java
index 3444b47..d805b92 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery1.java
@@ -36,7 +36,7 @@ public class TPCHQuery1 implements Program, ProgramDescription {
 
 	private static final long serialVersionUID = 1L;
 
-	private int degreeOfParallelism = 1;
+	private int parallelism = 1;
 	private String lineItemInputPath;
 	private String outputPath;
 	
@@ -45,28 +45,28 @@ public class TPCHQuery1 implements Program, ProgramDescription {
 		
 		
 		if (args.length != 3) {
-			this.degreeOfParallelism = 1;
+			this.parallelism = 1;
 			this.lineItemInputPath = "";
 			this.outputPath = "";
 		} else {
-			this.degreeOfParallelism = Integer.parseInt(args[0]);
+			this.parallelism = Integer.parseInt(args[0]);
 			this.lineItemInputPath = args[1];
 			this.outputPath = args[2];
 		}
 		
 		FileDataSource lineItems =
 			new FileDataSource(new IntTupleDataInFormat(), this.lineItemInputPath, "LineItems");
-		lineItems.setDegreeOfParallelism(this.degreeOfParallelism);
+		lineItems.setParallelism(this.parallelism);
 		
 		FileDataSink result = 
 			new FileDataSink(new StringTupleDataOutFormat(), this.outputPath, "Output");
-		result.setDegreeOfParallelism(this.degreeOfParallelism);
+		result.setParallelism(this.parallelism);
 		
 		MapOperator lineItemFilter = 
 			MapOperator.builder(new LineItemFilter())
 			.name("LineItem Filter")
 			.build();
-		lineItemFilter.setDegreeOfParallelism(this.degreeOfParallelism);
+		lineItemFilter.setParallelism(this.parallelism);
 		
 		ReduceOperator groupByReturnFlag = 
 			ReduceOperator.builder(new GroupByReturnFlag(), StringValue.class, 0)
@@ -82,6 +82,6 @@ public class TPCHQuery1 implements Program, ProgramDescription {
 
 	@Override
 	public String getDescription() {
-		return "Parameters: [dop] [lineitem-input] [output]";
+		return "Parameters: [parallelism] [lineitem-input] [output]";
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java
index 7a8ffc6..4bb0cdf 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery10.java
@@ -273,12 +273,12 @@ public class TPCHQuery10 implements Program, ProgramDescription {
 		final String nationsPath;
 		final String resultPath;
 		
-		final int degreeOfParallelism;
+		final int parallelism;
 
 		if (args.length < 6) {
 			throw new IllegalArgumentException("Invalid number of parameters");
 		} else {
-			degreeOfParallelism = Integer.parseInt(args[0]);
+			parallelism = Integer.parseInt(args[0]);
 			ordersPath = args[1];
 			lineitemsPath = args[2];
 			customersPath = args[3];
@@ -359,7 +359,7 @@ public class TPCHQuery10 implements Program, ProgramDescription {
 
 		// return the PACT plan
 		Plan p = new Plan(result, "TPCH Q10");
-		p.setDefaultParallelism(degreeOfParallelism);
+		p.setDefaultParallelism(parallelism);
 		return p;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java
index 2103747..ec3c5b4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery4.java
@@ -56,7 +56,7 @@ public class TPCHQuery4 implements Program, ProgramDescription {
 
 	private static Logger LOG = LoggerFactory.getLogger(TPCHQuery4.class);
 	
-	private int degreeOfParallelism = 1;
+	private int parallelism = 1;
 	private String ordersInputPath;
 	private String lineItemInputPath;
 	private String outputPath;
@@ -218,40 +218,40 @@ public class TPCHQuery4 implements Program, ProgramDescription {
 		
 		FileDataSource orders = 
 			new FileDataSource(new IntTupleDataInFormat(), this.ordersInputPath, "Orders");
-		orders.setDegreeOfParallelism(this.degreeOfParallelism);
+		orders.setParallelism(this.parallelism);
 		//orders.setOutputContract(UniqueKey.class);
 		
 		FileDataSource lineItems =
 			new FileDataSource(new IntTupleDataInFormat(), this.lineItemInputPath, "LineItems");
-		lineItems.setDegreeOfParallelism(this.degreeOfParallelism);
+		lineItems.setParallelism(this.parallelism);
 		
 		FileDataSink result = 
 				new FileDataSink(new StringTupleDataOutFormat(), this.outputPath, "Output");
-		result.setDegreeOfParallelism(degreeOfParallelism);
+		result.setParallelism(parallelism);
 		
 		MapOperator lineFilter = 
 				MapOperator.builder(LiFilter.class)
 			.name("LineItemFilter")
 			.build();
-		lineFilter.setDegreeOfParallelism(degreeOfParallelism);
+		lineFilter.setParallelism(parallelism);
 		
 		MapOperator ordersFilter = 
 				MapOperator.builder(OFilter.class)
 			.name("OrdersFilter")
 			.build();
-		ordersFilter.setDegreeOfParallelism(degreeOfParallelism);
+		ordersFilter.setParallelism(parallelism);
 		
 		JoinOperator join = 
 				JoinOperator.builder(JoinLiO.class, IntValue.class, 0, 0)
 			.name("OrdersLineitemsJoin")
 			.build();
-			join.setDegreeOfParallelism(degreeOfParallelism);
+			join.setParallelism(parallelism);
 		
 		ReduceOperator aggregation = 
 				ReduceOperator.builder(CountAgg.class, StringValue.class, 0)
 			.name("AggregateGroupBy")
 			.build();
-		aggregation.setDegreeOfParallelism(this.degreeOfParallelism);
+		aggregation.setParallelism(this.parallelism);
 		
 		lineFilter.setInput(lineItems);
 		ordersFilter.setInput(orders);
@@ -269,7 +269,7 @@ public class TPCHQuery4 implements Program, ProgramDescription {
 	 * @param args
 	 */
 	private void setArgs(String[] args) {
-		this.degreeOfParallelism = Integer.parseInt(args[0]);
+		this.parallelism = Integer.parseInt(args[0]);
 		this.ordersInputPath = args[1];
 		this.lineItemInputPath = args[2];
 		this.outputPath = args[3];
@@ -278,7 +278,7 @@ public class TPCHQuery4 implements Program, ProgramDescription {
 
 	@Override
 	public String getDescription() {
-		return "Parameters: [dop] [orders-input] [lineitem-input] [output]";
+		return "Parameters: [parallelism] [orders-input] [lineitem-input] [output]";
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java
index 925ed5c..c00d231 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQuery9.java
@@ -81,11 +81,11 @@ import org.slf4j.LoggerFactory;
  */
 @SuppressWarnings({"serial", "deprecation"})
 public class TPCHQuery9 implements Program, ProgramDescription {
-	public final String ARGUMENTS = "dop partInputPath partSuppInputPath ordersInputPath lineItemInputPath supplierInputPath nationInputPath outputPath";
+	public final String ARGUMENTS = "parallelism partInputPath partSuppInputPath ordersInputPath lineItemInputPath supplierInputPath nationInputPath outputPath";
 
 	private static Logger LOG = LoggerFactory.getLogger(TPCHQuery9.class);
 
-	private int degreeOfParallelism = 1;
+	private int parallelism = 1;
 
 	private String partInputPath, partSuppInputPath, ordersInputPath, lineItemInputPath, supplierInputPath,
 			nationInputPath;
@@ -100,7 +100,7 @@ public class TPCHQuery9 implements Program, ProgramDescription {
 		{
 			LOG.warn("number of arguments do not match!");
 			
-			this.degreeOfParallelism = 1;
+			this.parallelism = 1;
 			this.partInputPath = "";
 			this.partSuppInputPath = "";
 			this.ordersInputPath = "";
@@ -110,7 +110,7 @@ public class TPCHQuery9 implements Program, ProgramDescription {
 			this.outputPath = "";
 		}else
 		{
-			this.degreeOfParallelism = Integer.parseInt(args[0]);
+			this.parallelism = Integer.parseInt(args[0]);
 			this.partInputPath = args[1];
 			this.partSuppInputPath = args[2];
 			this.ordersInputPath = args[3];
@@ -240,7 +240,7 @@ public class TPCHQuery9 implements Program, ProgramDescription {
 		result.setInput(sumAmountAggregate);
 
 		Plan p = new Plan(result, "TPC-H query 9");
-		p.setDefaultParallelism(this.degreeOfParallelism);
+		p.setDefaultParallelism(this.parallelism);
 		return p;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java
index 415fde9..a681f64 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/TPCHQueryAsterix.java
@@ -147,7 +147,7 @@ public class TPCHQueryAsterix implements Program, ProgramDescription {
 		 */
 		// create DataSourceContract for Orders input
 		FileDataSource orders = new FileDataSource(new CsvInputFormat(), ordersPath, "Orders");
-		orders.setDegreeOfParallelism(numSubtasks);
+		orders.setParallelism(numSubtasks);
 		CsvInputFormat.configureRecordFormat(orders)
 			.recordDelimiter('\n')
 			.fieldDelimiter('|')
@@ -160,7 +160,7 @@ public class TPCHQueryAsterix implements Program, ProgramDescription {
 		 */
 		// create DataSourceContract for Customer input
 		FileDataSource customers = new FileDataSource(new CsvInputFormat(), customerPath, "Customers");
-		customers.setDegreeOfParallelism(numSubtasks);
+		customers.setParallelism(numSubtasks);
 		CsvInputFormat.configureRecordFormat(customers)
 			.recordDelimiter('\n')
 			.fieldDelimiter('|')
@@ -171,17 +171,17 @@ public class TPCHQueryAsterix implements Program, ProgramDescription {
 		JoinOperator joinCO = JoinOperator.builder(new JoinCO(), IntValue.class, 0, 0)
 			.name("JoinCO")
 			.build();
-		joinCO.setDegreeOfParallelism(numSubtasks);
+		joinCO.setParallelism(numSubtasks);
 
 		// create ReduceOperator for aggregating the result
 		ReduceOperator aggCO = ReduceOperator.builder(new AggCO(), StringValue.class, 1)
 			.name("AggCo")
 			.build();
-		aggCO.setDegreeOfParallelism(numSubtasks);
+		aggCO.setParallelism(numSubtasks);
 
 		// create DataSinkContract for writing the result
 		FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, "Output");
-		result.setDegreeOfParallelism(numSubtasks);
+		result.setParallelism(numSubtasks);
 		CsvOutputFormat.configureRecordFormat(result)
 			.recordDelimiter('\n')
 			.fieldDelimiter('|')

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java
index 45889ea..377c7ee 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/relational/WebLogAnalysis.java
@@ -318,7 +318,7 @@ public class WebLogAnalysis implements Program, ProgramDescription {
 
 		// Create DataSinkContract for writing the result of the OLAP query
 		FileDataSink result = new FileDataSink(new CsvOutputFormat(), output, antiJoinVisits, "Result");
-		result.setDegreeOfParallelism(numSubTasks);
+		result.setParallelism(numSubTasks);
 		CsvOutputFormat.configureRecordFormat(result)
 			.recordDelimiter('\n')
 			.fieldDelimiter('|')

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java
index 860e6b9..4563c07 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/sort/TeraSort.java
@@ -59,12 +59,12 @@ public final class TeraSort implements Program, ProgramDescription {
 		// This task will read the input data and generate the key/value pairs
 		final FileDataSource source = 
 				new FileDataSource(new TeraInputFormat(), input, "Data Source");
-		source.setDegreeOfParallelism(numSubTasks);
+		source.setParallelism(numSubTasks);
 
 		// This task writes the sorted data back to disk
 		final FileDataSink sink = 
 				new FileDataSink(new TeraOutputFormat(), output, "Data Sink");
-		sink.setDegreeOfParallelism(numSubTasks);
+		sink.setParallelism(numSubTasks);
 		sink.setGlobalOrder(new Ordering(0, TeraKey.class, Order.ASCENDING), new TeraDistribution());
 
 		sink.setInput(source);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
index 6f5e698..ec498ea 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java
@@ -174,7 +174,7 @@ public class ProcessFailureBatchRecoveryITCase {
 			final Throwable[] errorRef = new Throwable[1];
 
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort);
-			env.setDegreeOfParallelism(PARALLELISM);
+			env.setParallelism(PARALLELISM);
 			env.setNumberOfExecutionRetries(1);
 			env.getConfig().setExecutionMode(executionMode);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
index 48afce1..42b1c15 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
@@ -79,7 +79,7 @@ public class SimpleRecoveryITCase {
 				ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 						"localhost", cluster.getJobManagerRPCPort());
 
-				env.setDegreeOfParallelism(4);
+				env.setParallelism(4);
 				env.setNumberOfExecutionRetries(0);
 
 				env.generateSequence(1, 10)
@@ -108,7 +108,7 @@ public class SimpleRecoveryITCase {
 				ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 						"localhost", cluster.getJobManagerRPCPort());
 
-				env.setDegreeOfParallelism(4);
+				env.setParallelism(4);
 				env.setNumberOfExecutionRetries(0);
 
 				env.generateSequence(1, 10)
@@ -154,7 +154,7 @@ public class SimpleRecoveryITCase {
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 					"localhost", cluster.getJobManagerRPCPort());
 
-			env.setDegreeOfParallelism(4);
+			env.setParallelism(4);
 			env.setNumberOfExecutionRetries(1);
 
 			env.generateSequence(1, 10)
@@ -198,7 +198,7 @@ public class SimpleRecoveryITCase {
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 					"localhost", cluster.getJobManagerRPCPort());
 
-			env.setDegreeOfParallelism(4);
+			env.setParallelism(4);
 			env.setNumberOfExecutionRetries(3);
 
 			env.generateSequence(1, 10)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
index eb04234..aa92925 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
@@ -85,7 +85,7 @@ public class TaskManagerFailureRecoveryITCase {
 			final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 					"localhost", cluster.getJobManagerRPCPort());
 
-			env.setDegreeOfParallelism(PARALLELISM);
+			env.setParallelism(PARALLELISM);
 			env.setNumberOfExecutionRetries(1);
 
 			env.generateSequence(1, 10)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
index 007742b..1cdb736 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
@@ -31,7 +31,7 @@ public class ConsumePipelinedAndBlockingResultITCase extends JavaProgramTestBase
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(1);
+		env.setParallelism(1);
 
 		DataSet<Tuple1<Long>> pipelinedSource = env.fromElements(new Tuple1<Long>(1l));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
index 784e824..6d5e00e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/testjar/KMeansForTest.java
@@ -58,7 +58,7 @@ public class KMeansForTest implements Program {
 
 
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(4);
+		env.setParallelism(4);
 
 		// get input data
 		DataSet<Point> points = env.readCsvFile(pointsPath)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 33e0807..b6f045b 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala
@@ -106,7 +106,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * check correctness of groupReduce on tuples with key field selector and group sorting
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.get3TupleDataSet(env)
     val reduceDs =  ds.groupBy(1).sortGroup(2, Order.ASCENDING).reduceGroup {
       in =>
@@ -301,7 +301,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
 
     val env = ExecutionEnvironment.getExecutionEnvironment
     // important because it determines how often the combiner is called
-    env.setDegreeOfParallelism(2)
+    env.setParallelism(2)
     val ds =  CollectionDataSets.get3TupleDataSet(env)
 
     val reduceDs =  ds.groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine)
@@ -341,7 +341,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * check correctness of groupReduce with descending group sort
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.get3TupleDataSet(env)
     val reduceDs =  ds.groupBy(1).sortGroup(2, Order.DESCENDING).reduceGroup {
       in =>
@@ -386,7 +386,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * sorting
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.get3TupleDataSet(env).map { t =>
       MutableTuple3(t._1, t._2, t._3)
     }
@@ -478,7 +478,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * check correctness of groupReduce with descending group sort
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.get3TupleDataSet(env)
     val reduceDs =  ds.groupBy(1)
       .sortGroup("_3", Order.DESCENDING)
@@ -500,7 +500,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * Test int-based definition on group sort, for (full) nested Tuple
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.getGroupSortedNestedTupleDataSet(env)
     val reduceDs =  ds.groupBy("_2").sortGroup(0, Order.DESCENDING)
       .reduceGroup(new NestedTupleReducer)
@@ -515,7 +515,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * Test int-based definition on group sort, for (partial) nested Tuple ASC
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.getGroupSortedNestedTupleDataSet(env)
     val reduceDs =  ds.groupBy("_2")
       .sortGroup("_1._1", Order.ASCENDING)
@@ -532,7 +532,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * Test string-based definition on group sort, for (partial) nested Tuple DESC
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.getGroupSortedNestedTupleDataSet(env)
     val reduceDs =  ds.groupBy("_2")
       .sortGroup("_1._1", Order.DESCENDING)
@@ -549,7 +549,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * Test string-based definition on group sort, for two grouping keys
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.getGroupSortedNestedTupleDataSet(env)
     val reduceDs =  ds.groupBy("_2")
       .sortGroup("_1._1", Order.DESCENDING)
@@ -566,7 +566,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * Test string-based definition on group sort, for two grouping keys with Pojos
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env)
     val reduceDs =  ds.groupBy("hadoopFan")
       .sortGroup("theTuple._1", Order.DESCENDING)
@@ -601,7 +601,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * check correctness of sorted groupReduce on tuples with keyselector sorting
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.get3TupleDataSet(env)
     val reduceDs =  ds.groupBy(_._2).sortGroup(_._3, Order.DESCENDING).reduceGroup {
       in =>
@@ -658,7 +658,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * check correctness of sorted groupReduce with combine on tuples with keyselector sorting
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.get3TupleDataSet(env)
 
     val reduceDs =  ds.groupBy(_._2).sortGroup(_._3, Order.DESCENDING)
@@ -683,7 +683,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * check correctness of sorted groupReduceon with Tuple2 keyselector sorting
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.get5TupleDataSet(env)
 
     val reduceDs = ds.groupBy(_._1).sortGroup(t => (t._5, t._3), Order.DESCENDING).reduceGroup{
@@ -722,7 +722,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
      * Test grouping with pojo containing multiple pojos (was a bug)
      */
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     val ds =  CollectionDataSets.getPojoWithMultiplePojos(env)
     val reduceDs =  ds.groupBy("p2.a2")
       .reduceGroup {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 c5ced67..9bf3cce 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala
@@ -289,7 +289,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode)
       .where("nestedPojo.longNumber", "number", "str")
       .equalTo("_7", "_1", "_2")
     joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     env.execute()
     expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One," +
       "10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two," +
@@ -307,7 +307,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode)
     val joinDs = ds1.join(ds2).where("nestedPojo.longNumber", "number",
       "nestedTupleWithCustom._1").equalTo("_7", "_1", "_3")
     joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     env.execute()
     expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One," +
       "10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two," +
@@ -328,7 +328,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode)
         "nestedTupleWithCustom._2.myLong")
       .equalTo("_3", "_4", "_5")
     joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     env.execute()
     expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One," +
       "10000)\n" + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two," +
@@ -345,7 +345,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode)
     val ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env)
     val joinDs = ds1.join(ds2).where(0).equalTo("_1._1", "_1._2")
     joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     env.execute()
     expected = "((1,1),one),((1,1),one)\n" + "((2,2),two),((2,2),two)\n" + "((3,3),three),((3,3)," +
       "three)\n"
@@ -362,7 +362,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode)
     val ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env)
     val joinDs = ds1.join(ds2).where("_1._1").equalTo("_1._1")
     joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     env.execute()
     expected = "((1,1),one),((1,1),one)\n" + "((2,2),two),((2,2),two)\n" + "((3,3),three),((3,3)," +
       "three)\n"
@@ -378,7 +378,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode)
     val ds2 = CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env)
     val joinDs = ds1.join(ds2).where("*").equalTo("*")
     joinDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
     env.execute()
     expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" +
       "2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" +

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/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 8e00075..98bb446 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/PartitionITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/PartitionITCase.scala
@@ -103,20 +103,20 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(
     countsInPartition.writeAsText(resultPath, WriteMode.OVERWRITE)
     env.execute()
 
-    val numPerPartition : Int = 2220 / env.getDegreeOfParallelism / 10
+    val numPerPartition : Int = 2220 / env.getParallelism / 10
     expected = ""
-    for (i <- 0 until env.getDegreeOfParallelism) {
+    for (i <- 0 until env.getParallelism) {
       expected += "(" + i + "," + numPerPartition + ")\n"
     }
   }
 
   @Test
-  def testMapPartitionAfterRepartitionHasCorrectDOP(): Unit = {
+  def testMapPartitionAfterRepartitionHasCorrectParallelism(): Unit = {
     // Verify that mapPartition operation after repartition picks up correct
-    // DOP
+    // parallelism
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds = CollectionDataSets.get3TupleDataSet(env)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
 
     val unique = ds.partitionByHash(1)
       .setParallelism(4)
@@ -129,12 +129,12 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(
   }
 
   @Test
-  def testMapAfterRepartitionHasCorrectDOP(): Unit = {
+  def testMapAfterRepartitionHasCorrectParallelism(): Unit = {
     // Verify that map operation after repartition picks up correct
-    // DOP
+    // parallelism
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds = CollectionDataSets.get3TupleDataSet(env)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
 
     val count = ds.partitionByHash(0).setParallelism(4).map(
       new RichMapFunction[(Int, Long, String), Tuple1[Int]] {
@@ -157,12 +157,12 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(
   }
 
   @Test
-  def testFilterAfterRepartitionHasCorrectDOP(): Unit = {
+  def testFilterAfterRepartitionHasCorrectParallelism(): Unit = {
     // Verify that filter operation after repartition picks up correct
-    // DOP
+    // parallelism
     val env = ExecutionEnvironment.getExecutionEnvironment
     val ds = CollectionDataSets.get3TupleDataSet(env)
-    env.setDegreeOfParallelism(1)
+    env.setParallelism(1)
 
     val count = ds.partitionByHash(0).setParallelism(4).filter(
       new RichFilterFunction[(Int, Long, String)] {
@@ -188,7 +188,7 @@ class PartitionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(
   @Test
   def testPartitionNestedPojo(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setDegreeOfParallelism(3)
+    env.setParallelism(3)
     val ds = CollectionDataSets.getDuplicatePojoDataSet(env)
     val uniqLongs = ds
       .partitionByHash("nestedPojo.longNumber")

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
index ae3a20c..425cff6 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/AggregateTranslationTest.scala
@@ -31,9 +31,9 @@ class AggregateTranslationTest {
   @Test
   def translateAggregate(): Unit =  {
     try {
-      val DOP = 8
+      val parallelism = 8
 
-      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+      val env = ExecutionEnvironment.createLocalEnvironment(parallelism)
 
       val initialData = env.fromElements((3.141592, "foobar", 77L))
 
@@ -46,7 +46,7 @@ class AggregateTranslationTest {
 
       assertEquals(1, reducer.getKeyColumns(0).length)
       assertEquals(0, reducer.getKeyColumns(0)(0))
-      assertEquals(-1, reducer.getDegreeOfParallelism)
+      assertEquals(-1, reducer.getParallelism)
       assertTrue(reducer.isCombinable)
       assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala
index 969f970..11de8d5 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningTest.scala
@@ -37,7 +37,7 @@ class CustomPartitioningTest extends CompilerTestBase {
       val parallelism = 4
       
       val env = ExecutionEnvironment.getExecutionEnvironment
-      env.setDegreeOfParallelism(parallelism)
+      env.setParallelism(parallelism)
 
       val data = env.fromElements( (0,0) ).rebalance()
       
@@ -80,7 +80,7 @@ class CustomPartitioningTest extends CompilerTestBase {
       val parallelism = 4
       
       val env = ExecutionEnvironment.getExecutionEnvironment
-      env.setDegreeOfParallelism(parallelism)
+      env.setParallelism(parallelism)
       
       val data = env.fromElements( (0,0) ).rebalance()
       try {
@@ -106,7 +106,7 @@ class CustomPartitioningTest extends CompilerTestBase {
       val parallelism = 4
       
       val env = ExecutionEnvironment.getExecutionEnvironment
-      env.setDegreeOfParallelism(parallelism)
+      env.setParallelism(parallelism)
       
       val data = env.fromElements(new Pojo()).rebalance()
       
@@ -150,7 +150,7 @@ class CustomPartitioningTest extends CompilerTestBase {
       val parallelism = 4
       
       val env = ExecutionEnvironment.getExecutionEnvironment
-      env.setDegreeOfParallelism(parallelism)
+      env.setParallelism(parallelism)
       
       val data = env.fromElements(new Pojo()).rebalance()
       
@@ -177,7 +177,7 @@ class CustomPartitioningTest extends CompilerTestBase {
       val parallelism = 4
       
       val env = ExecutionEnvironment.getExecutionEnvironment
-      env.setDegreeOfParallelism(parallelism)
+      env.setParallelism(parallelism)
       
       val data = env.fromElements(new Pojo()).rebalance()
       

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
index 0246500..6aa4d75 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DeltaIterationTranslationTest.scala
@@ -46,11 +46,11 @@ class DeltaIterationTranslationTest {
       val AGGREGATOR_NAME = "AggregatorName"
       val ITERATION_KEYS = Array(2)
       val NUM_ITERATIONS = 13
-      val DEFAULT_DOP = 133
-      val ITERATION_DOP = 77
+      val DEFAULT_PARALLELISM = 133
+      val ITERATION_PARALLELISM = 77
 
       val env = ExecutionEnvironment.getExecutionEnvironment
-      env.setDegreeOfParallelism(DEFAULT_DOP)
+      env.setParallelism(DEFAULT_PARALLELISM)
 
       val initialSolutionSet = env.fromElements((3.44, 5L, "abc"))
       val initialWorkSet = env.fromElements((1.23, "abc"))
@@ -64,7 +64,7 @@ class DeltaIterationTranslationTest {
           (joined, joined.map(new NextWorksetMapper).name(BEFORE_NEXT_WORKSET_MAP))
       }
       result.name(ITERATION_NAME)
-        .setParallelism(ITERATION_DOP)
+        .setParallelism(ITERATION_PARALLELISM)
         .registerAggregator(AGGREGATOR_NAME, new LongSumAggregator)
 
       result.print()
@@ -72,7 +72,7 @@ class DeltaIterationTranslationTest {
 
       val p: Plan = env.createProgramPlan(JOB_NAME)
       assertEquals(JOB_NAME, p.getJobName)
-      assertEquals(DEFAULT_DOP, p.getDefaultParallelism)
+      assertEquals(DEFAULT_PARALLELISM, p.getDefaultParallelism)
       var sink1: GenericDataSinkBase[_] = null
       var sink2: GenericDataSinkBase[_] = null
       val sinks = p.getDataSinks.iterator
@@ -85,7 +85,7 @@ class DeltaIterationTranslationTest {
       assertEquals(iteration, sink2.getInput)
       assertEquals(NUM_ITERATIONS, iteration.getMaximumNumberOfIterations)
       assertArrayEquals(ITERATION_KEYS, iteration.getSolutionSetKeyFields)
-      assertEquals(ITERATION_DOP, iteration.getDegreeOfParallelism)
+      assertEquals(ITERATION_PARALLELISM, iteration.getParallelism)
       assertEquals(ITERATION_NAME, iteration.getName)
 
       val nextWorksetMapper: MapOperatorBase[_, _, _] =

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
index 79a584d..e97fc21 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/ReduceTranslationTest.scala
@@ -33,8 +33,8 @@ class ReduceTranslationTest {
   @Test
   def translateNonGroupedReduce(): Unit = {
     try {
-      val DOP = 8
-      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+      val parallelism = 8
+      val env = ExecutionEnvironment.createLocalEnvironment(parallelism)
 
       val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1)
 
@@ -50,7 +50,7 @@ class ReduceTranslationTest {
       assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getInputType)
       assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getOutputType)
       assertTrue(reducer.getKeyColumns(0) == null || reducer.getKeyColumns(0).length == 0)
-      assertTrue(reducer.getDegreeOfParallelism == 1 || reducer.getDegreeOfParallelism == -1)
+      assertTrue(reducer.getParallelism == 1 || reducer.getParallelism == -1)
       assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
     }
     catch {
@@ -65,8 +65,8 @@ class ReduceTranslationTest {
   @Test
   def translateGroupedReduceNoMapper(): Unit = {
     try {
-      val DOP: Int = 8
-      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+      val parallelism: Int = 8
+      val env = ExecutionEnvironment.createLocalEnvironment(parallelism)
 
       val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1)
 
@@ -78,7 +78,7 @@ class ReduceTranslationTest {
       val reducer: ReduceOperatorBase[_, _] = sink.getInput.asInstanceOf[ReduceOperatorBase[_, _]]
       assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getInputType)
       assertEquals(initialData.javaSet.getType, reducer.getOperatorInfo.getOutputType)
-      assertTrue(reducer.getDegreeOfParallelism == DOP || reducer.getDegreeOfParallelism == -1)
+      assertTrue(reducer.getParallelism == parallelism || reducer.getParallelism == -1)
       assertArrayEquals(Array[Int](2), reducer.getKeyColumns(0))
       assertTrue(reducer.getInput.isInstanceOf[GenericDataSourceBase[_, _]])
     }
@@ -94,8 +94,8 @@ class ReduceTranslationTest {
   @Test
   def translateGroupedReduceWithKeyExtractor(): Unit = {
     try {
-      val DOP: Int = 8
-      val env = ExecutionEnvironment.createLocalEnvironment(DOP)
+      val parallelism: Int = 8
+      val env = ExecutionEnvironment.createLocalEnvironment(parallelism)
 
       val initialData = env.fromElements((3.141592, "foobar", 77L)).setParallelism(1)
 
@@ -109,9 +109,9 @@ class ReduceTranslationTest {
         .asInstanceOf[PlanUnwrappingReduceOperator[_, _]]
       val keyExtractor: MapOperatorBase[_, _, _] = reducer.getInput
         .asInstanceOf[MapOperatorBase[_, _, _]]
-      assertEquals(1, keyExtractor.getDegreeOfParallelism)
-      assertEquals(4, reducer.getDegreeOfParallelism)
-      assertEquals(4, keyProjector.getDegreeOfParallelism)
+      assertEquals(1, keyExtractor.getParallelism)
+      assertEquals(4, reducer.getParallelism)
+      assertEquals(4, keyProjector.getParallelism)
       val keyValueInfo = new TupleTypeInfo(
         BasicTypeInfo.STRING_TYPE_INFO,
         createTypeInformation[(Double, String, Long)])

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
index ce17d72..9c3311b 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
@@ -170,7 +170,7 @@ object ApplicationMaster {
     if(slots != -1){
       output.println(s"${ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS}: $slots")
       output.println(
-        s"${ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY}: ${slots*taskManagerCount}")
+        s"${ConfigConstants.DEFAULT_PARALLELISM_KEY}: ${slots*taskManagerCount}")
     }
 
     output.println(s"${ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION}: " +


[4/9] flink git commit: [FLINK-1679] use a consistent name for parallelism

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java
index 7f468fa..87d7779 100644
--- a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java
@@ -178,8 +178,8 @@ public class ScalaAggregateOperator<IN> extends SingleInputOperator<IN, IN, Scal
 
 			// set input
 			po.setInput(input);
-			// set dop
-			po.setDegreeOfParallelism(this.getParallelism());
+			// set parallelism
+			po.setParallelism(this.getParallelism());
 
 			return po;
 		}
@@ -195,8 +195,8 @@ public class ScalaAggregateOperator<IN> extends SingleInputOperator<IN, IN, Scal
 
 			// set input
 			po.setInput(input);
-			// set dop
-			po.setDegreeOfParallelism(this.getParallelism());
+			// set parallelism
+			po.setParallelism(this.getParallelism());
 
 			SingleInputSemanticProperties props = new SingleInputSemanticProperties();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
index 00761ec..1291181 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
@@ -72,7 +72,7 @@ import scala.reflect.ClassTag
  * are named similarly. All functions are available in package
  * `org.apache.flink.api.common.functions`.
  *
- * The elements are partitioned depending on the degree of parallelism of the
+ * The elements are partitioned depending on the parallelism of the
  * [[ExecutionEnvironment]] or of one specific DataSet.
  *
  * Most of the operations have an implicit [[TypeInformation]] parameter. This is supplied by
@@ -149,13 +149,13 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
   }
 
   /**
-   * Sets the degree of parallelism of this operation. This must be greater than 1.
+   * Sets the parallelism of this operation. This must be greater than 1.
    */
-  def setParallelism(dop: Int) = {
+  def setParallelism(parallelism: Int) = {
     javaSet match {
-      case ds: DataSource[_] => ds.setParallelism(dop)
-      case op: Operator[_, _] => op.setParallelism(dop)
-      case di: DeltaIterationResultSet[_, _] => di.getIterationHead.parallelism(dop)
+      case ds: DataSource[_] => ds.setParallelism(parallelism)
+      case op: Operator[_, _] => op.setParallelism(parallelism)
+      case di: DeltaIterationResultSet[_, _] => di.getIterationHead.parallelism(parallelism)
       case _ =>
         throw new UnsupportedOperationException("Operator " + javaSet.toString + " cannot have " +
           "parallelism.")
@@ -164,7 +164,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
   }
 
   /**
-   * Returns the degree of parallelism of this operation.
+   * Returns the parallelism of this operation.
    */
   def getParallelism: Int = javaSet match {
     case ds: DataSource[_] => ds.getParallelism

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
index cccea78..4c1e627 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
@@ -78,9 +78,9 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
   }
 
   /**
-   * Sets the degree of parallelism (DOP) for operations executed through this environment.
-   * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with
-   * x parallel instances. This value can be overridden by specific operations using
+   * Sets the parallelism (parallelism) for operations executed through this environment.
+   * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
+   * with x parallel instances. This value can be overridden by specific operations using
    * [[DataSet.setParallelism]].
    * @deprecated Please use [[setParallelism]]
    */
@@ -90,8 +90,10 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
   }
 
   /**
-   * Returns the default degree of parallelism for this execution environment. Note that this
-   * value can be overridden by individual operations using [[DataSet.setParallelism]
+   * Sets the parallelism (parallelism) for operations executed through this environment.
+   * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
+   * with x parallel instances. This value can be overridden by specific operations using
+   * [[DataSet.setParallelism]].
    */
   def setParallelism(parallelism: Int): Unit = {
     javaEnv.setParallelism(parallelism)
@@ -432,7 +434,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
    * because the framework may move the elements into the cluster if needed.
    *
    * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a degree of parallelism of one.
+   * a parallelism of one.
    */
   def fromCollection[T: ClassTag : TypeInformation](
       data: Seq[T]): DataSet[T] = {
@@ -453,7 +455,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
    * framework might move into the cluster if needed.
    *
    * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a degree of parallelism of one.
+   * a parallelism of one.
    */
   def fromCollection[T: ClassTag : TypeInformation] (
     data: Iterator[T]): DataSet[T] = {
@@ -473,7 +475,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
    * same type and must be serializable.
    *
    * * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a degree of parallelism of one.
+   * a parallelism of one.
    */
   def fromElements[T: ClassTag : TypeInformation](data: T*): DataSet[T] = {
     Validate.notNull(data, "Data must not be null.")
@@ -610,10 +612,10 @@ object ExecutionEnvironment {
    * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads).
    */
   def createLocalEnvironment(
-      degreeOfParallelism: Int = Runtime.getRuntime.availableProcessors())
+      parallelism: Int = Runtime.getRuntime.availableProcessors())
       : ExecutionEnvironment = {
     val javaEnv = JavaEnv.createLocalEnvironment()
-    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+    javaEnv.setParallelism(parallelism)
     new ExecutionEnvironment(javaEnv)
   }
 
@@ -630,8 +632,8 @@ object ExecutionEnvironment {
   /**
    * Creates a remote execution environment. The remote environment sends (parts of) the program to
    * a cluster for execution. Note that all file paths used in the program must be accessible from
-   * the cluster. The execution will use the cluster's default degree of parallelism, unless the
-   * parallelism is set explicitly via [[ExecutionEnvironment.setDegreeOfParallelism()]].
+   * the cluster. The execution will use the cluster's default parallelism, unless the
+   * parallelism is set explicitly via [[ExecutionEnvironment.setParallelism()]].
    *
    * @param host The host name or address of the master (JobManager),
    *             where the program should be executed.
@@ -649,12 +651,12 @@ object ExecutionEnvironment {
   /**
    * Creates a remote execution environment. The remote environment sends (parts of) the program
    * to a cluster for execution. Note that all file paths used in the program must be accessible
-   * from the cluster. The execution will use the specified degree of parallelism.
+   * from the cluster. The execution will use the specified parallelism.
    *
    * @param host The host name or address of the master (JobManager),
    *             where the program should be executed.
    * @param port The port of the master (JobManager), where the program should be executed.
-   * @param degreeOfParallelism The degree of parallelism to use during the execution.
+   * @param parallelism The parallelism to use during the execution.
    * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the
    *                 program uses
    *                 user-defined functions, user-defined input formats, or any libraries,
@@ -664,10 +666,10 @@ object ExecutionEnvironment {
   def createRemoteEnvironment(
       host: String,
       port: Int,
-      degreeOfParallelism: Int,
+      parallelism: Int,
       jarFiles: String*): ExecutionEnvironment = {
     val javaEnv = JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*)
-    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+    javaEnv.setParallelism(parallelism)
     new ExecutionEnvironment(javaEnv)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
index 6eaa9ae..c54ee0c 100644
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
+++ b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
@@ -175,17 +175,17 @@ public class VertexCentricIteration<VertexKey extends Comparable<VertexKey> & Se
 	}
 	
 	/**
-	 * Sets the degree of parallelism for the iteration.
+	 * Sets the parallelism for the iteration.
 	 * 
-	 * @param parallelism The degree of parallelism.
+	 * @param parallelism The parallelism.
 	 */
 	public void setParallelism(int parallelism) {
-		Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default).");
+		Validate.isTrue(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default).");
 		this.parallelism = parallelism;
 	}
 	
 	/**
-	 * Gets the iteration's degree of parallelism.
+	 * Gets the iteration's parallelism.
 	 * 
 	 * @return The iterations parallelism, or -1, if not set.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
index 18826b6..6c195cb 100644
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
+++ b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
@@ -75,7 +75,7 @@ public class DegreesWithExceptionITCase {
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 				"localhost", cluster.getJobManagerRPCPort());
 
-		env.setDegreeOfParallelism(PARALLELISM);
+		env.setParallelism(PARALLELISM);
 
 		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
 				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
@@ -100,7 +100,7 @@ public class DegreesWithExceptionITCase {
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 				"localhost", cluster.getJobManagerRPCPort());
 
-		env.setDegreeOfParallelism(PARALLELISM);
+		env.setParallelism(PARALLELISM);
 
 		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
 				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
@@ -125,7 +125,7 @@ public class DegreesWithExceptionITCase {
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 				"localhost", cluster.getJobManagerRPCPort());
 
-		env.setDegreeOfParallelism(PARALLELISM);
+		env.setParallelism(PARALLELISM);
 
 		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
 				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
@@ -150,7 +150,7 @@ public class DegreesWithExceptionITCase {
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 				"localhost", cluster.getJobManagerRPCPort());
 
-		env.setDegreeOfParallelism(PARALLELISM);
+		env.setParallelism(PARALLELISM);
 
 		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
 				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
@@ -175,7 +175,7 @@ public class DegreesWithExceptionITCase {
 		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
 				"localhost", cluster.getJobManagerRPCPort());
 
-		env.setDegreeOfParallelism(PARALLELISM);
+		env.setParallelism(PARALLELISM);
 
 		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
 				TestGraphUtils.getLongLongEdgeInvalidSrcTrgData(env), env);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java
index b6650d2..bbb7503 100644
--- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java
+++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java
@@ -31,7 +31,7 @@ public class HadoopMapredITCase extends JavaProgramTestBase {
 	protected void preSubmit() throws Exception {
 		textPath = createTempFile("text.txt", WordCountData.TEXT);
 		resultPath = getTempDirPath("result");
-		this.setDegreeOfParallelism(4);
+		this.setParallelism(4);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java
index 7eee629..9b4aeea 100644
--- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java
+++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java
@@ -32,7 +32,7 @@ public class HadoopInputOutputITCase extends JavaProgramTestBase {
 	protected void preSubmit() throws Exception {
 		textPath = createTempFile("text.txt", WordCountData.TEXT);
 		resultPath = getTempDirPath("result");
-		this.setDegreeOfParallelism(4);
+		this.setParallelism(4);
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala
index 28fdfa6..95e2a25 100644
--- a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala
+++ b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/feature/PolynomialBaseITCase.scala
@@ -33,7 +33,7 @@ class PolynomialBaseITCase extends ShouldMatchers {
   def testMapElementToPolynomialVectorSpace (): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
 
-    env.setDegreeOfParallelism (2)
+    env.setParallelism (2)
 
     val input = Seq (
     LabeledVector (DenseVector (1), 1.0),
@@ -64,7 +64,7 @@ class PolynomialBaseITCase extends ShouldMatchers {
   def testMapVectorToPolynomialVectorSpace(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
 
-    env.setDegreeOfParallelism(2)
+    env.setParallelism(2)
 
     val input = Seq(
       LabeledVector(DenseVector(2, 3), 1.0),
@@ -96,7 +96,7 @@ class PolynomialBaseITCase extends ShouldMatchers {
   def testReturnEmptyVectorIfDegreeIsZero(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
 
-    env.setDegreeOfParallelism(2)
+    env.setParallelism(2)
 
     val input = Seq(
       LabeledVector(DenseVector(2, 3), 1.0),

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala
index 6e324cb..d783ecb 100644
--- a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala
+++ b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/recommendation/ALSITCase.scala
@@ -34,7 +34,7 @@ class ALSITCase extends ShouldMatchers {
 
     val env = ExecutionEnvironment.getExecutionEnvironment
 
-    env.setDegreeOfParallelism(2)
+    env.setParallelism(2)
 
     val als = ALS()
       .setIterations(iterations)

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala
index eb825b9..15292b7 100644
--- a/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala
+++ b/flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/regression/MultipleLinearRegressionITCase.scala
@@ -33,7 +33,7 @@ class MultipleLinearRegressionITCase extends ShouldMatchers {
   def testEstimationOfLinearFunction(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
 
-    env.setDegreeOfParallelism(2)
+    env.setParallelism(2)
 
     val learner = MultipleLinearRegression()
 
@@ -69,7 +69,7 @@ class MultipleLinearRegressionITCase extends ShouldMatchers {
   def testEstimationOfCubicFunction(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
 
-    env.setDegreeOfParallelism(2)
+    env.setParallelism(2)
 
     val polynomialBase = PolynomialBase()
     val learner = MultipleLinearRegression()

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
index 4f84467..d054975 100644
--- a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
+++ b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/VertexCentricIteration.java
@@ -217,17 +217,17 @@ public class VertexCentricIteration<VertexKey extends Comparable<VertexKey>, Ver
 	}
 	
 	/**
-	 * Sets the degree of parallelism for the iteration.
+	 * Sets the parallelism for the iteration.
 	 * 
-	 * @param parallelism The degree of parallelism.
+	 * @param parallelism The parallelism.
 	 */
 	public void setParallelism(int parallelism) {
-		Validate.isTrue(parallelism > 0 || parallelism == -1, "The degree of parallelism must be positive, or -1 (use default).");
+		Validate.isTrue(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default).");
 		this.parallelism = parallelism;
 	}
 	
 	/**
-	 * Gets the iteration's degree of parallelism.
+	 * Gets the iteration's parallelism.
 	 * 
 	 * @return The iterations parallelism, or -1, if not set.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java
index 019345f..38c26f0 100644
--- a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java
+++ b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelCompilerTest.java
@@ -48,7 +48,7 @@ public class SpargelCompilerTest extends CompilerTestBase {
 	public void testSpargelCompiler() {
 		try {
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+			env.setParallelism(DEFAULT_PARALLELISM);
 			// compose test program
 			{
 				DataSet<Long> vertexIds = env.generateSequence(1, 2);
@@ -116,7 +116,7 @@ public class SpargelCompilerTest extends CompilerTestBase {
 			
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+			env.setParallelism(DEFAULT_PARALLELISM);
 			// compose test program
 			{
 				DataSet<Long> bcVar = env.fromElements(1L);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
index b31618c..cd48dcd 100644
--- a/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
+++ b/flink-staging/flink-spargel/src/test/java/org/apache/flink/spargel/java/SpargelTranslationTest.java
@@ -49,7 +49,7 @@ public class SpargelTranslationTest {
 			;
 			final int NUM_ITERATIONS = 13;
 			
-			final int ITERATION_DOP = 77;
+			final int ITERATION_parallelism = 77;
 			
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -75,7 +75,7 @@ public class SpargelTranslationTest {
 				vertexIteration.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcUpdate);
 				
 				vertexIteration.setName(ITERATION_NAME);
-				vertexIteration.setParallelism(ITERATION_DOP);
+				vertexIteration.setParallelism(ITERATION_parallelism);
 				
 				vertexIteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
 				
@@ -93,7 +93,7 @@ public class SpargelTranslationTest {
 			// check the basic iteration properties
 			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
 			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
-			assertEquals(ITERATION_DOP, iteration.getParallelism());
+			assertEquals(ITERATION_parallelism, iteration.getParallelism());
 			assertEquals(ITERATION_NAME, iteration.getName());
 			
 			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());
@@ -129,7 +129,7 @@ public class SpargelTranslationTest {
 			;
 			final int NUM_ITERATIONS = 13;
 			
-			final int ITERATION_DOP = 77;
+			final int ITERATION_parallelism = 77;
 			
 			
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -154,7 +154,7 @@ public class SpargelTranslationTest {
 				vertexIteration.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcVar);
 				
 				vertexIteration.setName(ITERATION_NAME);
-				vertexIteration.setParallelism(ITERATION_DOP);
+				vertexIteration.setParallelism(ITERATION_parallelism);
 				
 				vertexIteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
 				
@@ -172,7 +172,7 @@ public class SpargelTranslationTest {
 			// check the basic iteration properties
 			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
 			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
-			assertEquals(ITERATION_DOP, iteration.getParallelism());
+			assertEquals(ITERATION_parallelism, iteration.getParallelism());
 			assertEquals(ITERATION_NAME, iteration.getName());
 			
 			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java
index 6a95b0c..1bf2962 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerExample.java
@@ -34,7 +34,7 @@ public class KafkaConsumerExample {
 			return;
 		}
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setDegreeOfParallelism(4);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4);
 
 		DataStream<String> kafkaStream = env
 				.addSource(new KafkaSource<String>(host + ":" + port, topic, new JavaDefaultStringSchema()));

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java
index e7abf11..b1e0f0b 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExample.java
@@ -36,7 +36,7 @@ public class KafkaProducerExample {
 			return;
 		}
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setDegreeOfParallelism(4);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4);
 
 		@SuppressWarnings({ "unused", "serial" })
 		DataStream<String> stream1 = env.addSource(new SourceFunction<String>() {

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java
index 2fd31ed..d43460b 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSimpleConsumerExample.java
@@ -34,7 +34,7 @@ public class KafkaSimpleConsumerExample {
 			return;
 		}
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setDegreeOfParallelism(4);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment().setParallelism(4);
 		DataStream<String> kafkaStream = env
 				.addSource(new PersistentKafkaSource<String>(host + ":" + port, topic, new JavaDefaultStringSchema()));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
index a094b89..95609f9 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
@@ -183,7 +183,7 @@ public class KafkaITCase {
 		stream.addSink(new KafkaSink<String>(zookeeperConnectionString, TOPIC, new JavaDefaultStringSchema()));
 
 		try {
-			env.setDegreeOfParallelism(1);
+			env.setParallelism(1);
 			env.execute();
 		} catch (JobExecutionException good) {
 			Throwable t = good.getCause();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
index a6eade8..a9ae77a 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -404,7 +404,7 @@ public class ConnectedDataStream<IN1, IN2> {
 
 		dataStream1.streamGraph.addCoTask(returnStream.getId(), functionInvokable,
 				getInputType1(), getInputType2(), outTypeInfo, functionName,
-				environment.getDegreeOfParallelism());
+				environment.getParallelism());
 
 		dataStream1.connectGraph(dataStream1, returnStream.getId(), 1);
 		dataStream1.connectGraph(dataStream2, returnStream.getId(), 2);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 5768101..59ef108 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -103,7 +103,7 @@ public class DataStream<OUT> {
 	protected final StreamExecutionEnvironment environment;
 	protected final Integer id;
 	protected final String type;
-	protected int degreeOfParallelism;
+	protected int parallelism;
 	protected List<String> userDefinedNames;
 	protected StreamPartitioner<OUT> partitioner;
 	@SuppressWarnings("rawtypes")
@@ -133,7 +133,7 @@ public class DataStream<OUT> {
 		this.id = counter;
 		this.type = operatorType;
 		this.environment = environment;
-		this.degreeOfParallelism = environment.getDegreeOfParallelism();
+		this.parallelism = environment.getParallelism();
 		this.streamGraph = environment.getStreamGraph();
 		this.userDefinedNames = new ArrayList<String>();
 		this.partitioner = new DistributePartitioner<OUT>(true);
@@ -152,7 +152,7 @@ public class DataStream<OUT> {
 		this.environment = dataStream.environment;
 		this.id = dataStream.id;
 		this.type = dataStream.type;
-		this.degreeOfParallelism = dataStream.degreeOfParallelism;
+		this.parallelism = dataStream.parallelism;
 		this.userDefinedNames = new ArrayList<String>(dataStream.userDefinedNames);
 		this.partitioner = dataStream.partitioner;
 		this.streamGraph = dataStream.streamGraph;
@@ -177,12 +177,12 @@ public class DataStream<OUT> {
 	}
 
 	/**
-	 * Gets the degree of parallelism for this operator.
+	 * Gets the parallelism for this operator.
 	 * 
 	 * @return The parallelism set for this operator.
 	 */
 	public int getParallelism() {
-		return this.degreeOfParallelism;
+		return this.parallelism;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
index 0dda976..b8e0a7d 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
@@ -41,11 +41,11 @@ public class DataStreamSource<OUT> extends SingleOutputStreamOperator<OUT, DataS
 	}
 
 	@Override
-	public DataStreamSource<OUT> setParallelism(int dop) {
-		if (dop > 1 && !isParallel) {
+	public DataStreamSource<OUT> setParallelism(int parallelism) {
+		if (parallelism > 1 && !isParallel) {
 			throw new IllegalArgumentException("Source: " + this.id + " is not a parallel source");
 		} else {
-			return (DataStreamSource<OUT>) super.setParallelism(dop);
+			return (DataStreamSource<OUT>) super.setParallelism(parallelism);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
index 7832777..2ac60fe 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
@@ -216,7 +216,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 			return out;
 		} else if (transformation == WindowTransformation.REDUCEWINDOW
 				&& parallelism != discretizedStream.getExecutionEnvironment()
-						.getDegreeOfParallelism()) {
+						.getParallelism()) {
 			DiscretizedStream<OUT> out = transform(transformation, "Window partitioner", getType(),
 					new WindowPartitioner<OUT>(parallelism)).setParallelism(parallelism);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index 16284d4..8ffe1fc 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -62,20 +62,20 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	}
 
 	/**
-	 * Sets the degree of parallelism for this operator. The degree must be 1 or
+	 * Sets the parallelism for this operator. The degree must be 1 or
 	 * more.
 	 * 
-	 * @param dop
-	 *            The degree of parallelism for this operator.
-	 * @return The operator with set degree of parallelism.
+	 * @param parallelism
+	 *            The parallelism for this operator.
+	 * @return The operator with set parallelism.
 	 */
-	public SingleOutputStreamOperator<OUT, O> setParallelism(int dop) {
-		if (dop < 1) {
+	public SingleOutputStreamOperator<OUT, O> setParallelism(int parallelism) {
+		if (parallelism < 1) {
 			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
 		}
-		this.degreeOfParallelism = dop;
+		this.parallelism = parallelism;
 
-		streamGraph.setParallelism(id, degreeOfParallelism);
+		streamGraph.setParallelism(id, parallelism);
 
 		return this;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
index efbbda9..784d20c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
@@ -394,7 +394,7 @@ public class WindowedDataStream<OUT> {
 	}
 
 	/**
-	 * Returns the degree of parallelism for the stream discretizer. The
+	 * Returns the parallelism for the stream discretizer. The
 	 * returned parallelism is either 1 for for non-parallel global policies (or
 	 * when the input stream is non-parallel), environment parallelism for the
 	 * policies that can run in parallel (such as, any ditributed policy, reduce
@@ -408,7 +408,7 @@ public class WindowedDataStream<OUT> {
 		return isLocal
 				|| (transformation == WindowTransformation.REDUCEWINDOW && WindowUtils
 						.isParallelPolicy(getTrigger(), getEviction(), dataStream.getParallelism()))
-				|| (discretizerKey != null) ? dataStream.environment.getDegreeOfParallelism() : 1;
+				|| (discretizerKey != null) ? dataStream.environment.getParallelism() : 1;
 
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
index 4824fca..e1b1453 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
@@ -29,7 +29,7 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 	 */
 	@Override
 	public void execute() throws Exception {
-		ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(), getDegreeOfParallelism());
+		ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(), getParallelism());
 	}
 
 	/**
@@ -42,6 +42,6 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 	@Override
 	public void execute(String jobName) throws Exception {
 		ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(jobName),
-				getDegreeOfParallelism());
+				getParallelism());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 2eb05ad..3142bdd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -120,8 +120,8 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 
 	@Override
 	public String toString() {
-		return "Remote Environment (" + this.host + ":" + this.port + " - DOP = "
-				+ (getDegreeOfParallelism() == -1 ? "default" : getDegreeOfParallelism()) + ")";
+		return "Remote Environment (" + this.host + ":" + this.port + " - parallelism = "
+				+ (getParallelism() == -1 ? "default" : getParallelism()) + ")";
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
index 7d41d2a..7ae78f1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
@@ -32,15 +32,15 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment {
 	protected List<File> jars;
 	protected Client client;
 
-	protected StreamContextEnvironment(Client client, List<File> jars, int dop) {
+	protected StreamContextEnvironment(Client client, List<File> jars, int parallelism) {
 		this.client = client;
 		this.jars = jars;
-		if (dop > 0) {
-			setDegreeOfParallelism(dop);
+		if (parallelism > 0) {
+			setParallelism(parallelism);
 		} else {
-			setDegreeOfParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY,
-					ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE));
+			setParallelism(GlobalConfiguration.getInteger(
+					ConfigConstants.DEFAULT_PARALLELISM_KEY,
+					ConfigConstants.DEFAULT_PARALLELISM));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index dc1826b..9f2ccff 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -64,7 +64,7 @@ import org.apache.flink.streaming.api.invokable.StreamInvokable;
  */
 public abstract class StreamExecutionEnvironment {
 
-	private static int defaultLocalDop = Runtime.getRuntime().availableProcessors();
+	private static int defaultLocalParallelism = Runtime.getRuntime().availableProcessors();
 
 	private long bufferTimeout = 100;
 
@@ -93,9 +93,9 @@ public abstract class StreamExecutionEnvironment {
 	}
 
 	/**
-	 * Gets the degree of parallelism with which operation are executed by
+	 * Gets the parallelism with which operation are executed by
 	 * default. Operations can individually override this value to use a
-	 * specific degree of parallelism.
+	 * specific parallelism.
 	 * 
 	 * @return The parallelism used by operations, unless they
 	 *         override that value.
@@ -252,12 +252,12 @@ public abstract class StreamExecutionEnvironment {
 	 * Sets the default parallelism that will be used for the local execution
 	 * environment created by {@link #createLocalEnvironment()}.
 	 * 
-	 * @param degreeOfParallelism
-	 *            The degree of parallelism to use as the default local
+	 * @param parallelism
+	 *            The parallelism to use as the default local
 	 *            parallelism.
 	 */
-	public static void setDefaultLocalParallelism(int degreeOfParallelism) {
-		defaultLocalDop = degreeOfParallelism;
+	public static void setDefaultLocalParallelism(int parallelism) {
+		defaultLocalParallelism = parallelism;
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -617,7 +617,7 @@ public abstract class StreamExecutionEnvironment {
 		}
 
 		boolean isParallel = function instanceof ParallelSourceFunction;
-		int dop = isParallel ? getDegreeOfParallelism() : 1;
+		int parallelism = isParallel ? getParallelism() : 1;
 
 		ClosureCleaner.clean(function, true);
 		StreamInvokable<OUT, OUT> sourceInvokable = new SourceInvokable<OUT>(function);
@@ -626,7 +626,7 @@ public abstract class StreamExecutionEnvironment {
 				outTypeInfo, sourceInvokable, isParallel);
 
 		streamGraph.addSourceVertex(returnStream.getId(), sourceInvokable, null, outTypeInfo,
-				sourceName, dop);
+				sourceName, parallelism);
 
 		return returnStream;
 	}
@@ -652,7 +652,7 @@ public abstract class StreamExecutionEnvironment {
 		if (env instanceof ContextEnvironment) {
 			ContextEnvironment ctx = (ContextEnvironment) env;
 			currentEnvironment = createContextEnvironment(ctx.getClient(), ctx.getJars(),
-					ctx.getDegreeOfParallelism());
+					ctx.getParallelism());
 		} else if (env instanceof OptimizerPlanEnvironment | env instanceof PreviewPlanEnvironment) {
 			currentEnvironment = new StreamPlanEnvironment(env);
 		} else {
@@ -662,38 +662,38 @@ public abstract class StreamExecutionEnvironment {
 	}
 
 	private static StreamExecutionEnvironment createContextEnvironment(Client client,
-			List<File> jars, int dop) {
-		return new StreamContextEnvironment(client, jars, dop);
+			List<File> jars, int parallelism) {
+		return new StreamContextEnvironment(client, jars, parallelism);
 	}
 
 	/**
 	 * Creates a {@link LocalStreamEnvironment}. The local execution environment
 	 * will run the program in a multi-threaded fashion in the same JVM as the
-	 * environment was created in. The default degree of parallelism of the
+	 * environment was created in. The default parallelism of the
 	 * local environment is the number of hardware contexts (CPU cores /
 	 * threads), unless it was specified differently by
-	 * {@link #setDegreeOfParallelism(int)}.
+	 * {@link #setParallelism(int)}.
 	 * 
 	 * @return A local execution environment.
 	 */
 	public static LocalStreamEnvironment createLocalEnvironment() {
-		return createLocalEnvironment(defaultLocalDop);
+		return createLocalEnvironment(defaultLocalParallelism);
 	}
 
 	/**
 	 * Creates a {@link LocalStreamEnvironment}. The local execution environment
 	 * will run the program in a multi-threaded fashion in the same JVM as the
-	 * environment was created in. It will use the degree of parallelism
+	 * environment was created in. It will use the parallelism
 	 * specified in the parameter.
 	 * 
-	 * @param degreeOfParallelism
-	 *            The degree of parallelism for the local environment.
-	 * @return A local execution environment with the specified degree of
+	 * @param parallelism
+	 *            The parallelism for the local environment.
+	 * @return A local execution environment with the specified
 	 *         parallelism.
 	 */
-	public static LocalStreamEnvironment createLocalEnvironment(int degreeOfParallelism) {
+	public static LocalStreamEnvironment createLocalEnvironment(int parallelism) {
 		currentEnvironment = new LocalStreamEnvironment();
-		currentEnvironment.setDegreeOfParallelism(degreeOfParallelism);
+		currentEnvironment.setParallelism(parallelism);
 		return (LocalStreamEnvironment) currentEnvironment;
 	}
 
@@ -703,7 +703,7 @@ public abstract class StreamExecutionEnvironment {
 	 * (parts of) the program to a cluster for execution. Note that all file
 	 * paths used in the program must be accessible from the cluster. The
 	 * execution will use no parallelism, unless the parallelism is set
-	 * explicitly via {@link #setDegreeOfParallelism}.
+	 * explicitly via {@link #setParallelism}.
 	 * 
 	 * @param host
 	 *            The host name or address of the master (JobManager), where the
@@ -728,7 +728,7 @@ public abstract class StreamExecutionEnvironment {
 	 * Creates a {@link RemoteStreamEnvironment}. The remote environment sends
 	 * (parts of) the program to a cluster for execution. Note that all file
 	 * paths used in the program must be accessible from the cluster. The
-	 * execution will use the specified degree of parallelism.
+	 * execution will use the specified parallelism.
 	 * 
 	 * @param host
 	 *            The host name or address of the master (JobManager), where the
@@ -736,8 +736,8 @@ public abstract class StreamExecutionEnvironment {
 	 * @param port
 	 *            The port of the master (JobManager), where the program should
 	 *            be executed.
-	 * @param degreeOfParallelism
-	 *            The degree of parallelism to use during the execution.
+	 * @param parallelism
+	 *            The parallelism to use during the execution.
 	 * @param jarFiles
 	 *            The JAR files with code that needs to be shipped to the
 	 *            cluster. If the program uses user-defined functions,
@@ -746,9 +746,9 @@ public abstract class StreamExecutionEnvironment {
 	 * @return A remote environment that executes the program on a cluster.
 	 */
 	public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
-			int degreeOfParallelism, String... jarFiles) {
+			int parallelism, String... jarFiles) {
 		currentEnvironment = new RemoteStreamEnvironment(host, port, jarFiles);
-		currentEnvironment.setDegreeOfParallelism(degreeOfParallelism);
+		currentEnvironment.setParallelism(parallelism);
 		return currentEnvironment;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
index 1cff7e7..2cf5cc2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
@@ -32,13 +32,13 @@ public class StreamPlanEnvironment extends StreamExecutionEnvironment {
 		super();
 		this.env = env;
 
-		int dop = env.getDegreeOfParallelism();
-		if (dop > 0) {
-			setDegreeOfParallelism(dop);
+		int parallelism = env.getParallelism();
+		if (parallelism > 0) {
+			setParallelism(parallelism);
 		} else {
-			setDegreeOfParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE_KEY,
-					ConfigConstants.DEFAULT_PARALLELIZATION_DEGREE));
+			setParallelism(GlobalConfiguration.getInteger(
+					ConfigConstants.DEFAULT_PARALLELISM_KEY,
+					ConfigConstants.DEFAULT_PARALLELISM));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
index 9ff8a7f..947f8ab 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/sink/PrintSinkFunction.java
@@ -68,7 +68,7 @@ public class PrintSinkFunction<IN> extends RichSinkFunction<IN> {
 		// get the target stream
 		stream = target == STD_OUT ? System.out : System.err;
 		
-		// set the prefix if we have a >1 DOP
+		// set the prefix if we have a >1 parallelism
 		prefix = (context.getNumberOfParallelSubtasks() > 1) ? 
 				((context.getIndexOfThisSubtask() + 1) + "> ") : null;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
index ec8c226..0a423cc 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/SourceFunction.java
@@ -26,7 +26,7 @@ import org.apache.flink.util.Collector;
  * Interface for a stream data source.
  *
  * <p>Sources implementing this specific interface are executed with
- * degree of parallelism 1. To execute your sources in parallel
+ * parallelism 1. To execute your sources in parallel
  * see {@link ParallelSourceFunction}.</p>
  *
  * @param <OUT> The type of the records produced by this source.

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
index 691b111..d04e7e6 100755
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
@@ -37,12 +37,12 @@ public class ClusterUtil {
 	 * 
 	 * @param jobGraph
 	 *            jobGraph
-	 * @param degreeOfParallelism
+	 * @param parallelism
 	 *            numberOfTaskTrackers
 	 * @param memorySize
 	 *            memorySize
 	 */
-	public static void runOnMiniCluster(JobGraph jobGraph, int degreeOfParallelism, long memorySize)
+	public static void runOnMiniCluster(JobGraph jobGraph, int parallelism, long memorySize)
 			throws Exception {
 
 		Configuration configuration = jobGraph.getJobConfiguration();
@@ -50,7 +50,7 @@ public class ClusterUtil {
 		LocalFlinkMiniCluster exec = null;
 
 		configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize);
-		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, degreeOfParallelism);
+		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
 		if (LOG.isInfoEnabled()) {
 			LOG.info("Running on mini cluster");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
index d853846..0446b61 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
@@ -27,11 +27,11 @@ import org.junit.Test;
 
 public abstract class StreamingProgramTestBase extends AbstractTestBase {
 
-	private static final int DEFAULT_DEGREE_OF_PARALLELISM = 4;
+	private static final int DEFAULT_PARALLELISM = 4;
 
 	private JobExecutionResult latestExecutionResult;
 
-	private int degreeOfParallelism = DEFAULT_DEGREE_OF_PARALLELISM;
+	private int parallelism = DEFAULT_PARALLELISM;
 
 
 	public StreamingProgramTestBase() {
@@ -40,16 +40,16 @@ public abstract class StreamingProgramTestBase extends AbstractTestBase {
 
 	public StreamingProgramTestBase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(degreeOfParallelism);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
-	public void setDegreeOfParallelism(int degreeOfParallelism) {
-		this.degreeOfParallelism = degreeOfParallelism;
-		setTaskManagerNumSlots(degreeOfParallelism);
+	public void setParallelism(int parallelism) {
+		this.parallelism = parallelism;
+		setTaskManagerNumSlots(parallelism);
 	}
 	
-	public int getDegreeOfParallelism() {
-		return degreeOfParallelism;
+	public int getParallelism() {
+		return parallelism;
 	}
 	
 	public JobExecutionResult getLatestExecutionResult() {
@@ -86,7 +86,7 @@ public abstract class StreamingProgramTestBase extends AbstractTestBase {
 			}
 
 			// prepare the test environment
-			TestStreamEnvironment env = new TestStreamEnvironment(this.executor, this.degreeOfParallelism);
+			TestStreamEnvironment env = new TestStreamEnvironment(this.executor, this.parallelism);
 			env.setAsContext();
 
 			// call the test program

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
index 6e0821d..5e785f9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
@@ -38,15 +38,15 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 	private ForkableFlinkMiniCluster executor;
 	private boolean internalExecutor;
 
-	public TestStreamEnvironment(int degreeOfParallelism, long memorySize){
-		setDegreeOfParallelism(degreeOfParallelism);
+	public TestStreamEnvironment(int parallelism, long memorySize){
+		setParallelism(parallelism);
 		this.memorySize = memorySize;
 		internalExecutor = true;
 	}
 
-	public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int dop){
+	public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism){
 		this.executor = executor;
-		setDefaultLocalParallelism(dop);
+		setDefaultLocalParallelism(parallelism);
 	}
 
 	@Override
@@ -62,7 +62,7 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 			Configuration configuration = jobGraph.getJobConfiguration();
 
 			configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-					getDegreeOfParallelism());
+					getParallelism());
 			configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize);
 
 			executor = new ForkableFlinkMiniCluster(configuration);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 59f86b8..3db6531 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -57,11 +57,11 @@ class DataStream[T](javaStream: JavaStream[T]) {
   def getJavaStream: JavaStream[T] = javaStream
 
   /**
-   * Sets the degree of parallelism of this operation. This must be greater than 1.
+   * Sets the parallelism of this operation. This must be greater than 1.
    */
-  def setParallelism(dop: Int): DataStream[T] = {
+  def setParallelism(parallelism: Int): DataStream[T] = {
     javaStream match {
-      case ds: SingleOutputStreamOperator[_, _] => ds.setParallelism(dop)
+      case ds: SingleOutputStreamOperator[_, _] => ds.setParallelism(parallelism)
       case _ =>
         throw new UnsupportedOperationException("Operator " + javaStream.toString +  " cannot " +
           "have " +
@@ -71,7 +71,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
   }
 
   /**
-   * Returns the degree of parallelism of this operation.
+   * Returns the parallelism of this operation.
    */
   def getParallelism: Int = javaStream match {
     case op: SingleOutputStreamOperator[_, _] => op.getParallelism

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 598b590..cfa7c18 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -33,9 +33,9 @@ import org.apache.flink.streaming.api.function.source.FileMonitoringFunction.Wat
 class StreamExecutionEnvironment(javaEnv: JavaEnv) {
 
   /**
-   * Sets the degree of parallelism (DOP) for operations executed through this environment.
-   * Setting a DOP of x here will cause all operators (such as join, map, reduce) to run with
-   * x parallel instances. This value can be overridden by specific operations using
+   * Sets the parallelism for operations executed through this environment.
+   * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
+   * with x parallel instances. This value can be overridden by specific operations using
    * [[DataStream.setParallelism]].
    * @deprecated Please use [[setParallelism]]
    */
@@ -239,7 +239,7 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
    * same type and must be serializable.
    *
    * * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a degree of parallelism of one.
+   * a parallelism of one.
    */
   def fromElements[T: ClassTag: TypeInformation](data: T*): DataStream[T] = {
     val typeInfo = implicitly[TypeInformation[T]]
@@ -251,7 +251,7 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
    * because the framework may move the elements into the cluster if needed.
    *
    * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a degree of parallelism of one.
+   * a parallelism of one.
    */
   def fromCollection[T: ClassTag: TypeInformation](
     data: Seq[T]): DataStream[T] = {
@@ -352,16 +352,16 @@ object StreamExecutionEnvironment {
    * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads).
    */
   def createLocalEnvironment(
-    degreeOfParallelism: Int =  Runtime.getRuntime.availableProcessors()):
+    parallelism: Int =  Runtime.getRuntime.availableProcessors()):
   StreamExecutionEnvironment = {
-    new StreamExecutionEnvironment(JavaEnv.createLocalEnvironment(degreeOfParallelism))
+    new StreamExecutionEnvironment(JavaEnv.createLocalEnvironment(parallelism))
   }
 
   /**
    * Creates a remote execution environment. The remote environment sends (parts of) the program to
    * a cluster for execution. Note that all file paths used in the program must be accessible from
-   * the cluster. The execution will use the cluster's default degree of parallelism, unless the
-   * parallelism is set explicitly via [[StreamExecutionEnvironment.setDegreeOfParallelism()]].
+   * the cluster. The execution will use the cluster's default parallelism, unless the
+   * parallelism is set explicitly via [[StreamExecutionEnvironment.setParallelism()]].
    *
    * @param host The host name or address of the master (JobManager),
    *             where the program should be executed.
@@ -380,12 +380,12 @@ object StreamExecutionEnvironment {
   /**
    * Creates a remote execution environment. The remote environment sends (parts of) the program
    * to a cluster for execution. Note that all file paths used in the program must be accessible
-   * from the cluster. The execution will use the specified degree of parallelism.
+   * from the cluster. The execution will use the specified parallelism.
    *
    * @param host The host name or address of the master (JobManager),
    *             where the program should be executed.
    * @param port The port of the master (JobManager), where the program should be executed.
-   * @param degreeOfParallelism The degree of parallelism to use during the execution.
+   * @param parallelism The parallelism to use during the execution.
    * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the
    *                 program uses
    *                 user-defined functions, user-defined input formats, or any libraries,
@@ -395,10 +395,10 @@ object StreamExecutionEnvironment {
   def createRemoteEnvironment(
     host: String,
     port: Int,
-    degreeOfParallelism: Int,
+    parallelism: Int,
     jarFiles: String*): StreamExecutionEnvironment = {
     val javaEnv = JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*)
-    javaEnv.setDegreeOfParallelism(degreeOfParallelism)
+    javaEnv.setParallelism(parallelism)
     new StreamExecutionEnvironment(javaEnv)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java
index 9396b66..fd328ae 100644
--- a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java
+++ b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/HDFSTest.java
@@ -41,7 +41,7 @@ import java.io.StringWriter;
 
 /**
  * This test should logically be located in the 'flink-runtime' tests. However, this project
- * has already all dependencies required (flink-java-examples). Also, the DOPOneExecEnv is here.
+ * has already all dependencies required (flink-java-examples). Also, the ParallelismOneExecEnv is here.
  */
 public class HDFSTest {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java
index e1235b6..3b2fb7f 100644
--- a/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java
+++ b/flink-staging/flink-tachyon/src/test/java/org/apache/flink/tachyon/TachyonFileSystemWrapperTest.java
@@ -128,7 +128,7 @@ public class TachyonFileSystemWrapperTest {
 			addHDConfToFlinkConf.setString(ConfigConstants.HDFS_DEFAULT_CONFIG, HADOOP_CONFIG_PATH.toString());
 			GlobalConfiguration.includeConfiguration(addHDConfToFlinkConf);
 
-			new DopOneTestEnvironment(); // initialize DOP one
+			new DopOneTestEnvironment(); // initialize parallelism one
 
 			WordCount.main(new String[]{input, output});
 
@@ -157,7 +157,7 @@ public class TachyonFileSystemWrapperTest {
 				@Override
 				public ExecutionEnvironment createExecutionEnvironment() {
 					LocalEnvironment le = new LocalEnvironment();
-					le.setDegreeOfParallelism(1);
+					le.setParallelism(1);
 					return le;
 				}
 			});

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java
index 788327a..435713b 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java
@@ -69,10 +69,10 @@ public abstract class CompilerTestBase {
 	public void setup() {
 		this.dataStats = new DataStatistics();
 		this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator());
-		this.withStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
+		this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
 		
 		this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator());
-		this.noStatsCompiler.setDefaultDegreeOfParallelism(DEFAULT_PARALLELISM);
+		this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
 	}
 	
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
index 6e86896..2214000 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
@@ -29,11 +29,11 @@ import org.apache.flink.api.java.tuple.Tuple;
 
 public abstract class JavaProgramTestBase extends AbstractTestBase {
 
-	private static final int DEFAULT_DEGREE_OF_PARALLELISM = 4;
+	private static final int DEFAULT_PARALLELISM = 4;
 	
 	private JobExecutionResult latestExecutionResult;
 	
-	private int degreeOfParallelism = DEFAULT_DEGREE_OF_PARALLELISM;
+	private int parallelism = DEFAULT_PARALLELISM;
 
 	/**
 	 * The number of times a test should be repeated.
@@ -42,7 +42,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 	 * tests repeatedly might help to discover resource leaks, race conditions etc.
 	 */
 	private int numberOfTestRepetitions = 1;
-	
+
 	private boolean isCollectionExecution;
 
 	public JavaProgramTestBase() {
@@ -51,20 +51,20 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 	
 	public JavaProgramTestBase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(degreeOfParallelism);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
-	public void setDegreeOfParallelism(int degreeOfParallelism) {
-		this.degreeOfParallelism = degreeOfParallelism;
-		setTaskManagerNumSlots(degreeOfParallelism);
+	public void setParallelism(int parallelism) {
+		this.parallelism = parallelism;
+		setTaskManagerNumSlots(parallelism);
 	}
 
 	public void setNumberOfTestRepetitions(int numberOfTestRepetitions) {
 		this.numberOfTestRepetitions = numberOfTestRepetitions;
 	}
 	
-	public int getDegreeOfParallelism() {
-		return isCollectionExecution ? 1 : degreeOfParallelism;
+	public int getParallelism() {
+		return isCollectionExecution ? 1 : parallelism;
 	}
 	
 	public JobExecutionResult getLatestExecutionResult() {
@@ -110,7 +110,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 			}
 			
 			// prepare the test environment
-			TestEnvironment env = new TestEnvironment(this.executor, this.degreeOfParallelism);
+			TestEnvironment env = new TestEnvironment(this.executor, this.parallelism);
 			env.getConfig().enableObjectReuse();
 			env.setAsContext();
 
@@ -162,7 +162,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 			}
 
 			// prepare the test environment
-			TestEnvironment env = new TestEnvironment(this.executor, this.degreeOfParallelism);
+			TestEnvironment env = new TestEnvironment(this.executor, this.parallelism);
 			env.getConfig().disableObjectReuse();
 			env.setAsContext();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
index 67a4797..eafe9ad 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
@@ -35,7 +35,7 @@ import org.junit.Test;
 
 public abstract class RecordAPITestBase extends AbstractTestBase {
 
-	protected static final int DOP = 4;
+	protected static final int parallelism = 4;
 	
 	protected JobExecutionResult jobExecutionResult;
 	
@@ -48,7 +48,7 @@ public abstract class RecordAPITestBase extends AbstractTestBase {
 	
 	public RecordAPITestBase(Configuration config) {
 		super(config);
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
index 44f35e7..02c1434 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
@@ -39,9 +39,9 @@ public class TestEnvironment extends ExecutionEnvironment {
 	protected JobExecutionResult latestResult;
 
 
-	public TestEnvironment(ForkableFlinkMiniCluster executor, int degreeOfParallelism) {
+	public TestEnvironment(ForkableFlinkMiniCluster executor, int parallelism) {
 		this.executor = executor;
-		setDegreeOfParallelism(degreeOfParallelism);
+		setParallelism(parallelism);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
index cd38418..1d7d206 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
@@ -83,7 +83,7 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 		
 		Assert.assertEquals(Integer.valueOf(3), (Integer) res.getAccumulatorResult("num-lines"));
 
-		Assert.assertEquals(Double.valueOf(getDegreeOfParallelism()), (Double)res.getAccumulatorResult("open-close-counter"));
+		Assert.assertEquals(Double.valueOf(getParallelism()), (Double)res.getAccumulatorResult("open-close-counter"));
 		
 		// Test histogram (words per line distribution)
 		Map<Integer, Integer> dist = Maps.newHashMap();

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
index daf5181..4868aff 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
@@ -95,7 +95,7 @@ public class BroadcastBranchingITCase extends RecordAPITestBase {
 		MapOperator mp2 = MapOperator.builder(Mp2.class).setBroadcastVariable("z", mp1).input(jn2).build();
 
 		FileDataSink output = new FileDataSink(new ContractITCaseOutputFormat(), resultPath);
-		output.setDegreeOfParallelism(1);
+		output.setParallelism(1);
 		output.setInput(mp2);
 
 		return new Plan(output);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
index 06c7506..41d24b8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
@@ -40,7 +40,7 @@ public class BroadcastVarInitializationITCase extends JavaProgramTestBase {
 	protected void testProgram() throws Exception {
 		
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setDegreeOfParallelism(4);
+		env.setParallelism(4);
 		
 		DataSet<Integer> data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8);
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
index e9374b1..34d4133 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
@@ -67,7 +67,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 
 	private static final int NUM_FEATURES = 3;
 
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
 	protected String pointsPath;
 
@@ -76,7 +76,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	protected String resultPath;
 
 	public BroadcastVarsNepheleITCase(){
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 
@@ -131,7 +131,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
-		return createJobGraphV1(this.pointsPath, this.modelsPath, this.resultPath, DOP);
+		return createJobGraphV1(this.pointsPath, this.modelsPath, this.resultPath, parallelism);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
index 8e13408..2406d6e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -60,9 +60,9 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	
 	private static final int MEMORY_PER_CONSUMER = 2;
 
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
-	private static final double MEMORY_FRACTION_PER_CONSUMER = (double)MEMORY_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*DOP;
+	private static final double MEMORY_FRACTION_PER_CONSUMER = (double)MEMORY_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*parallelism;
 
 	protected String dataPath;
 	protected String clusterPath;
@@ -70,7 +70,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 
 	
 	public KMeansIterativeNepheleITCase() {
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 	@Override
@@ -87,7 +87,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 
 	@Override
 	protected JobGraph getJobGraph() throws Exception {
-		return createJobGraph(dataPath, clusterPath, this.resultPath, DOP, 20);
+		return createJobGraph(dataPath, clusterPath, this.resultPath, parallelism, 20);
 	}
 
 	// -------------------------------------------------------------------------------------------------------------
@@ -252,8 +252,8 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return tail;
 	}
 	
-	private static AbstractJobVertex createSync(JobGraph jobGraph, int numIterations, int dop) {
-		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, dop);
+	private static AbstractJobVertex createSync(JobGraph jobGraph, int numIterations, int parallelism) {
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.setIterationId(ITERATION_ID);

http://git-wip-us.apache.org/repos/asf/flink/blob/cf84bca1/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
index 9bbd282..36db34d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
@@ -33,10 +33,10 @@ import org.apache.flink.util.Collector;
 
 @SuppressWarnings("deprecation")
 public class MapCancelingITCase extends CancellingTestBase {
-	private static final int DOP = 4;
+	private static final int parallelism = 4;
 
 	public MapCancelingITCase() {
-		setTaskManagerNumSlots(DOP);
+		setTaskManagerNumSlots(parallelism);
 	}
 	
 //	@Test
@@ -51,7 +51,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 5 * 1000, 10 * 1000);
 	}
@@ -68,7 +68,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 5 * 1000, 10 * 1000);
 	}
@@ -85,7 +85,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 10 * 1000, 10 * 1000);
 	}
@@ -102,7 +102,7 @@ public class MapCancelingITCase extends CancellingTestBase {
 		
 		
 		Plan p = new Plan(sink);
-		p.setDefaultParallelism(DOP);
+		p.setDefaultParallelism(parallelism);
 		
 		runAndCancelJob(p, 10 * 1000, 10 * 1000);
 	}


[9/9] flink git commit: [FLINK-1679] extend faq and programming guide to clarify parallelism

Posted by mx...@apache.org.
[FLINK-1679] extend faq and programming guide to clarify parallelism


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/126f9f79
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/126f9f79
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/126f9f79

Branch: refs/heads/master
Commit: 126f9f799071688fe80955a7e7cfa991f53c95af
Parents: 013ed82
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 18 10:44:45 2015 +0100
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Mar 23 09:03:56 2015 +0100

----------------------------------------------------------------------
 docs/config.md            | 16 +++++++++----
 docs/faq.md               | 21 ++++++++++++++++
 docs/programming_guide.md | 54 ++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 86 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/126f9f79/docs/config.md
----------------------------------------------------------------------
diff --git a/docs/config.md b/docs/config.md
index 09a01d9..3524a60 100644
--- a/docs/config.md
+++ b/docs/config.md
@@ -80,11 +80,17 @@ This value is typically proportional to the number of physical CPU cores that
 the TaskManager's machine has (e.g., equal to the number of cores, or half the
 number of cores). [More about task slots](config.html#configuring-taskmanager-processing-slots).
 
-- `parallelism.default`: The default parallelism to use for
-programs that have no parallelism specified. (DEFAULT: 1). For
-setups that have no concurrent jobs running, setting this value to
-NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all
-available execution resources for the program's execution.
+- `parallelism.default`: The default parallelism to use for programs that have
+no parallelism specified. (DEFAULT: 1). For setups that have no concurrent jobs
+running, setting this value to NumTaskManagers * NumSlotsPerTaskManager will
+cause the system to use all available execution resources for the program's
+execution. **Note**: The default parallelism can be overwriten for an entire
+job by calling `setParallelism(int parallelism)` on the `ExecutionEnvironment`
+or by passing `-p <parallelism>` to the Flink Command-line frontend. It can be
+overwritten for single transformations by calling `setParallelism(int
+parallelism)` on an operator. See the [programming
+guide](programming_guide.html#parallel-execution) for more information about the
+parallelism.
 
 - `fs.hdfs.hadoopconf`: The absolute path to the Hadoop File System's (HDFS)
 configuration directory (OPTIONAL VALUE).

http://git-wip-us.apache.org/repos/asf/flink/blob/126f9f79/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
index 2c3fe53..5c31bc6 100644
--- a/docs/faq.md
+++ b/docs/faq.md
@@ -85,6 +85,27 @@ tracking the behavior of the parallel execution. They allow you to gather
 information inside the program's operations and show them after the program
 execution.
 
+### What is the parallelism? How do I set it?
+
+In Flink programs, the parallelism determines how operations are split into
+individual tasks which are assigned to task slots. Each node in a cluster has at
+least one task slot. The total number of task slots is the number of all task slots
+on all machines. If the parallelism is set to `N`, Flink tries to divide an
+operation into `N` parallel tasks which can be computed concurrently using the
+available task slots. The number of task slots should be equal to the
+parallelism to ensure that all tasks can be computed in a task slot concurrently.
+
+**Note**: Not all operations can be divided into multiple tasks. For example, a
+`GroupReduce` operation without a grouping has to be performed with a
+parallelism of 1 because the entire group needs to be present at exactly one
+node to perform the reduce operation. Flink will determine whether the
+parallelism has to be 1 and set it accordingly.
+
+The parallelism can be set in numerous ways to ensure a fine-grained control
+over the execution of a Flink program. See
+[Configuration](config.html#common-options) for detailed instructions on how to
+set the parallelism.
+
 ## Errors
 
 ### Why am I getting a "NonSerializableException" ?

http://git-wip-us.apache.org/repos/asf/flink/blob/126f9f79/docs/programming_guide.md
----------------------------------------------------------------------
diff --git a/docs/programming_guide.md b/docs/programming_guide.md
index 1750a12..4e15b91 100644
--- a/docs/programming_guide.md
+++ b/docs/programming_guide.md
@@ -2789,6 +2789,60 @@ env.execute("Word Count Example")
 </div>
 </div>
 
+### Client Level
+
+The parallelism can be set at the Client when submitting jobs to Flink. The
+Client can either be a Java or a Scala program. One example of such a Client is
+Flink's Command-line Interface (CLI).
+
+For the CLI client, the parallelism parameter can be specified with `-p`. For
+exampple:
+
+    ./bin/flink run -p 10 ../examples/*WordCount-java*.jar
+
+
+In a Java/Scala program, the parallelism is set as follows:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+
+try {
+    PackagedProgram program = new PackagedProgram(file, args);
+    InetSocketAddress jobManagerAddress = RemoteExecutor.getInetFromHostport("localhost:6123");
+    Configuration config = new Configuration();
+
+    Client client = new Client(jobManagerAddress, config, program.getUserCodeClassLoader());
+
+    // set the parallelism to 10 here
+    client.run(program, 10, true);
+
+} catch (ProgramInvocationException e) {
+    e.printStackTrace();
+}
+
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+try {
+    PackagedProgram program = new PackagedProgram(file, args)
+    InetSocketAddress jobManagerAddress = RemoteExecutor.getInetFromHostport("localhost:6123")
+    Configuration config = new Configuration()
+
+    Client client = new Client(jobManagerAddress, new Configuration(), program.getUserCodeClassLoader())
+
+    // set the parallelism to 10 here
+    client.run(program, 10, true)
+
+} catch {
+    case e: Exception => e.printStackTrace
+}
+{% endhighlight %}
+</div>
+</div>
+
+
 ### System Level
 
 A system-wide default parallelism for all execution environments can be defined by setting the


[8/9] flink git commit: [FLINK-1679] deprecate old parallelism config entry

Posted by mx...@apache.org.
[FLINK-1679] deprecate old parallelism config entry

old config parameter can still be used

OLD
parallelization.degree.default

NEW
parallelism.default


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/013ed82f
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/013ed82f
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/013ed82f

Branch: refs/heads/master
Commit: 013ed82ff3eccc0946d20a955d98524a7ca0f7e4
Parents: cf84bca
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 18 10:44:44 2015 +0100
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Mar 23 09:03:56 2015 +0100

----------------------------------------------------------------------
 .../org/apache/flink/configuration/ConfigConstants.java     | 9 ++++++++-
 .../src/main/java/org/apache/flink/optimizer/Optimizer.java | 9 +++++++--
 .../streaming/api/environment/StreamContextEnvironment.java | 7 ++++++-
 .../streaming/api/environment/StreamPlanEnvironment.java    | 7 ++++++-
 4 files changed, 27 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/013ed82f/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index b472d8a..09f55fd 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -34,7 +34,14 @@ public final class ConfigConstants {
 	 * The config parameter defining the default parallelism for jobs.
 	 */
 	public static final String DEFAULT_PARALLELISM_KEY = "parallelism.default";
-	
+
+	/**
+	 * The deprecated config parameter defining the default parallelism for jobs.
+	 */
+	@Deprecated
+	public static final String DEFAULT_PARALLELISM_KEY_OLD = "parallelization.degree.default";
+
+
 	/**
 	 * Config parameter for the number of re-tries for failed tasks. Setting this
 	 * value to 0 effectively disables fault tolerance.

http://git-wip-us.apache.org/repos/asf/flink/blob/013ed82f/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
index 90421b7..c80cfc2 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
@@ -348,10 +348,15 @@ public class Optimizer {
 		this.costEstimator = estimator;
 
 		// determine the default parallelism
+		// check for old key string first, then for new one
 		this.defaultParallelism = GlobalConfiguration.getInteger(
-				ConfigConstants.DEFAULT_PARALLELISM_KEY,
+				ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
 				ConfigConstants.DEFAULT_PARALLELISM);
-		
+		// now check for new one which overwrites old values
+		this.defaultParallelism = GlobalConfiguration.getInteger(
+				ConfigConstants.DEFAULT_PARALLELISM_KEY,
+				this.defaultParallelism);
+
 		if (defaultParallelism < 1) {
 			LOG.warn("Config value " + defaultParallelism + " for option "
 					+ ConfigConstants.DEFAULT_PARALLELISM + " is invalid. Ignoring and using a value of 1.");

http://git-wip-us.apache.org/repos/asf/flink/blob/013ed82f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
index 7ae78f1..f7dd0bf 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
@@ -38,9 +38,14 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment {
 		if (parallelism > 0) {
 			setParallelism(parallelism);
 		} else {
+			// first check for old parallelism config key
 			setParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELISM_KEY,
+					ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
 					ConfigConstants.DEFAULT_PARALLELISM));
+			// then for new
+			setParallelism(GlobalConfiguration.getInteger(
+					ConfigConstants.DEFAULT_PARALLELISM_KEY,
+					getParallelism()));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/013ed82f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
index 2cf5cc2..592fa1a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
@@ -36,9 +36,14 @@ public class StreamPlanEnvironment extends StreamExecutionEnvironment {
 		if (parallelism > 0) {
 			setParallelism(parallelism);
 		} else {
+			// first check for old parallelism config key
 			setParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELISM_KEY,
+					ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
 					ConfigConstants.DEFAULT_PARALLELISM));
+			// then for new
+			setParallelism(GlobalConfiguration.getInteger(
+					ConfigConstants.DEFAULT_PARALLELISM_KEY,
+					getParallelism()));
 		}
 	}