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

[1/4] flink git commit: [FLINK-1771] Add support for submitting single jobs to a detached YARN session

Repository: flink
Updated Branches:
  refs/heads/master 121a5a0c7 -> 6b0d40764


http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java
index 7562c7c..ce5e93f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/RelationalQueryCompilerTest.java
@@ -32,8 +32,8 @@ import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
-import org.apache.flink.test.compiler.util.OperatorResolver;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.OperatorResolver;
 import org.apache.flink.test.recordJobs.relational.TPCHQuery3;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java
index d52d898..8cdf1b4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/WordCountCompilerTest.java
@@ -38,7 +38,7 @@ import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.test.recordJobs.wordcount.WordCount;
 import org.apache.flink.test.recordJobs.wordcount.WordCount.CountWords;
 import org.apache.flink.test.recordJobs.wordcount.WordCount.TokenizeLine;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java
index b38b784..3e127b9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsCoGroupTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.test.recordJobs.graph.ConnectedComponentsWithCoGroup;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsTest.java
index dcc9c15..9fd81fd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/ConnectedComponentsTest.java
@@ -44,7 +44,7 @@ import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents;
 import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.DuplicateLongMap;
 import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java
index bf2bbae..4275755 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/IterativeKMeansTest.java
@@ -36,8 +36,8 @@ import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
-import org.apache.flink.test.compiler.util.OperatorResolver;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.OperatorResolver;
 import org.apache.flink.test.recordJobs.kmeans.KMeansBroadcast;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
index e6a1e69..27c1644 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
index f05bd25..4edd68e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/iterations/PageRankCompilerTest.java
@@ -37,7 +37,7 @@ import org.apache.flink.examples.java.graph.PageRankBasic.JoinVertexWithEdgesMat
 import org.apache.flink.examples.java.graph.PageRankBasic.RankAssigner;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Assert;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
index 5c12c0f..4bb6cfc 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/plandump/DumpCompiledPlanTest.java
@@ -24,7 +24,7 @@ import org.apache.flink.client.program.PackagedProgram.PreviewPlanEnvironment;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.examples.java.clustering.KMeans;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.test.recordJobs.graph.DeltaPageRankWithInitialDeltas;
 import org.apache.flink.test.recordJobs.kmeans.KMeansBroadcast;
 import org.apache.flink.test.recordJobs.kmeans.KMeansSingleStep;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 0d2c469..19cf611 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
@@ -88,7 +88,7 @@ public class TaskFailureITCase extends FailingTestBase {
 		plan.setDefaultParallelism(parallelism);
 
 		// optimize and compile plan 
-		Optimizer pc = new Optimizer(new DataStatistics());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.config);
 		OptimizedPlan op = pc.compile(plan);
 		
 		// return job graph of failing job
@@ -118,7 +118,7 @@ public class TaskFailureITCase extends FailingTestBase {
 		plan.setDefaultParallelism(4);
 
 		// optimize and compile plan
-		Optimizer pc = new Optimizer(new DataStatistics());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.config);
 		OptimizedPlan op = pc.compile(plan);
 
 		// return job graph of working job

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 498a638..c268e74 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
@@ -133,7 +133,7 @@ public class ReduceITCase extends RecordAPITestBase {
 
 		Plan plan = new Plan(output);
 
-		Optimizer pc = new Optimizer(new DataStatistics());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.config);
 		OptimizedPlan op = pc.compile(plan);
 
 		JobGraphGenerator jgg = new JobGraphGenerator();

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 c214dbd..3148383 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
@@ -124,7 +124,7 @@ public class UnionSinkITCase extends RecordAPITestBase {
 		Plan plan = new Plan(output);
 		plan.setDefaultParallelism(parallelism);
 
-		Optimizer pc = new Optimizer(new DataStatistics());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.config);
 		OptimizedPlan op = pc.compile(plan);
 
 		JobGraphGenerator jgg = new JobGraphGenerator();

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/scala/org/apache/flink/api/scala/compiler/PartitionOperatorTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/compiler/PartitionOperatorTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/compiler/PartitionOperatorTranslationTest.scala
index ddb0dc7..3fefa01 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/compiler/PartitionOperatorTranslationTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/compiler/PartitionOperatorTranslationTest.scala
@@ -18,11 +18,11 @@
 
 package org.apache.flink.api.scala.compiler
 
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.junit.Test
 import org.junit.Assert._
 import org.apache.flink.api.scala._
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
-import org.apache.flink.test.compiler.util.CompilerTestBase
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
 import org.apache.flink.api.common.functions.Partitioner
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupCustomPartitioningTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupCustomPartitioningTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupCustomPartitioningTest.scala
index 6eb9c85..8d75f2e 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupCustomPartitioningTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupCustomPartitioningTest.scala
@@ -18,11 +18,11 @@
 
 package org.apache.flink.api.scala.operators.translation
 
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.junit.Assert._
 import org.junit.Test
 import org.apache.flink.api.common.functions.Partitioner
 import org.apache.flink.api.scala._
-import org.apache.flink.test.compiler.util.CompilerTestBase
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
 import org.apache.flink.api.common.operators.Order

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala
index 8ae8918..8d816ee 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala
@@ -18,11 +18,11 @@
 
 package org.apache.flink.api.scala.operators.translation
 
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.junit.Assert._
 import org.junit.Test
 import org.apache.flink.api.common.functions.Partitioner
 import org.apache.flink.api.scala._
-import org.apache.flink.test.compiler.util.CompilerTestBase
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
 import org.apache.flink.api.common.operators.Order

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingKeySelectorTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingKeySelectorTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingKeySelectorTest.scala
index 072130a..395f36a 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingKeySelectorTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingKeySelectorTest.scala
@@ -18,13 +18,13 @@
 
 package org.apache.flink.api.scala.operators.translation
 
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.junit.Assert._
 import org.junit.Test
 import org.apache.flink.api.scala._
 import org.apache.flink.api.common.functions.Partitioner
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
-import org.apache.flink.test.compiler.util.CompilerTestBase
 import org.apache.flink.api.common.operators.Order
 import org.apache.flink.api.common.InvalidProgramException
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingPojoTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingPojoTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingPojoTest.scala
index 59c5b37..a02d2af 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingPojoTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingPojoTest.scala
@@ -18,13 +18,13 @@
 
 package org.apache.flink.api.scala.operators.translation
 
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.junit.Assert._
 import org.junit.Test
 import org.apache.flink.api.scala._
 import org.apache.flink.api.common.functions.Partitioner
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
-import org.apache.flink.test.compiler.util.CompilerTestBase
 import scala.collection.immutable.Seq
 import org.apache.flink.api.common.operators.Order
 import org.apache.flink.api.common.InvalidProgramException

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingTupleTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingTupleTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingTupleTest.scala
index 1f183b8..25efe48 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingTupleTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingTupleTest.scala
@@ -23,7 +23,7 @@ import org.junit.Test
 
 import org.apache.flink.api.common.functions.Partitioner
 import org.apache.flink.api.scala._
-import org.apache.flink.test.compiler.util.CompilerTestBase
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
 import org.apache.flink.api.common.operators.Order

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 11de8d5..fe30376 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
@@ -19,10 +19,10 @@
 package org.apache.flink.api.scala.operators.translation
 
 import org.apache.flink.api.scala._
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.junit.Test
 import org.junit.Assert._
 import org.apache.flink.api.common.functions.Partitioner
-import org.apache.flink.test.compiler.util.CompilerTestBase
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
 import org.apache.flink.api.common.InvalidProgramException

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/JoinCustomPartitioningTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/JoinCustomPartitioningTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/JoinCustomPartitioningTest.scala
index 1caa686..2467596 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/JoinCustomPartitioningTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/JoinCustomPartitioningTest.scala
@@ -18,11 +18,11 @@
 
 package org.apache.flink.api.scala.operators.translation
 
+import org.apache.flink.optimizer.util.CompilerTestBase
 import org.junit.Assert._
 import org.junit.Test
 import org.apache.flink.api.common.functions.Partitioner
 import org.apache.flink.api.scala._
-import org.apache.flink.test.compiler.util.CompilerTestBase
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType
 import org.apache.flink.optimizer.plan.SingleInputPlanNode
 import org.apache.flink.api.common.operators.Order

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index ff58cf0..81db49d 100644
--- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -55,7 +55,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 						"-n", "1",
 						"-jm", "512",
 						"-tm", "1024", "-qu", "qa-team"},
-				"Number of connected TaskManagers changed to 1. Slots available: 1", RunTypes.YARN_SESSION);
+				"Number of connected TaskManagers changed to 1. Slots available: 1", null, RunTypes.YARN_SESSION);
 	}
 
 
@@ -66,11 +66,11 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	@Test
 	public void testNonexistingQueue() {
 		addTestAppender(FlinkYarnClient.class, Level.WARN);
-		runWithArgs(new String[] {"-j", flinkUberjar.getAbsolutePath(),
+		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 				"-n", "1",
 				"-jm", "512",
 				"-tm", "1024",
-				"-qu", "doesntExist"}, "to unknown queue: doesntExist", RunTypes.YARN_SESSION);
+				"-qu", "doesntExist"}, "to unknown queue: doesntExist", null, RunTypes.YARN_SESSION);
 		checkForLogString("The specified queue 'doesntExist' does not exist. Available queues: default, qa-team");
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index 4e2bed7..3949b8c 100644
--- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -18,6 +18,7 @@
 package org.apache.flink.yarn;
 
 import com.google.common.base.Joiner;
+import org.apache.commons.io.FileUtils;
 import org.apache.flink.client.FlinkYarnSessionCli;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
@@ -49,7 +50,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
+import java.io.FilenameFilter;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -93,8 +98,10 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 						"-n", "1",
 						"-jm", "512",
-						"-tm", "1024"},
-				"Number of connected TaskManagers changed to 1. Slots available: 1", RunTypes.YARN_SESSION);
+						"-tm", "1024",
+						"-s", "2" // Test that 2 slots are started on the TaskManager.
+				},
+				"Number of connected TaskManagers changed to 1. Slots available: 2",null, RunTypes.YARN_SESSION);
 		LOG.info("Finished testClientStartup()");
 	}
 
@@ -284,7 +291,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	@Test
 	public void testQueryCluster() {
 		LOG.info("Starting testQueryCluster()");
-		runWithArgs(new String[] {"-q"}, "Summary: totalMemory 8192 totalCores 1332", RunTypes.YARN_SESSION); // we have 666*2 cores.
+		runWithArgs(new String[] {"-q"}, "Summary: totalMemory 8192 totalCores 1332",null, RunTypes.YARN_SESSION); // we have 666*2 cores.
 		LOG.info("Finished testQueryCluster()");
 	}
 
@@ -299,29 +306,11 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 				"-n", "1",
 				"-jm", "512",
 				"-tm", "1024",
-				"-qu", "doesntExist"}, "Number of connected TaskManagers changed to 1. Slots available: 1", RunTypes.YARN_SESSION);
+				"-qu", "doesntExist"}, "Number of connected TaskManagers changed to 1. Slots available: 1", null, RunTypes.YARN_SESSION);
 		LOG.info("Finished testNonexistingQueue()");
 	}
 
 	/**
-	 * Test requesting more resources than available.
-	 */
-	@Test
-	public void testMoreNodesThanAvailable() {
-		if(ignoreOnTravis()) {
-			return;
-		}
-		addTestAppender(FlinkYarnClient.class, Level.WARN);
-		LOG.info("Starting testMoreNodesThanAvailable()");
-		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
-				"-n", "10",
-				"-jm", "512",
-				"-tm", "1024"}, "Number of connected TaskManagers changed to", RunTypes.YARN_SESSION); // the number of TMs depends on the speed of the test hardware
-		LOG.info("Finished testMoreNodesThanAvailable()");
-		checkForLogString("This YARN session requires 10752MB of memory in the cluster. There are currently only 8192MB available.");
-	}
-
-	/**
 	 * The test cluster has the following resources:
 	 * - 2 Nodes with 4096 MB each.
 	 * - RM_SCHEDULER_MINIMUM_ALLOCATION_MB is 512
@@ -343,7 +332,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 				"-n", "5",
 				"-jm", "256",
-				"-tm", "1585"}, "Number of connected TaskManagers changed to", RunTypes.YARN_SESSION);
+				"-tm", "1585"}, "Number of connected TaskManagers changed to", null, RunTypes.YARN_SESSION);
 		LOG.info("Finished testResourceComputation()");
 		checkForLogString("This YARN session requires 8437MB of memory in the cluster. There are currently only 8192MB available.");
 	}
@@ -373,7 +362,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 				"-n", "2",
 				"-jm", "256",
-				"-tm", "3840"}, "Number of connected TaskManagers changed to", RunTypes.YARN_SESSION);
+				"-tm", "3840"}, "Number of connected TaskManagers changed to", null, RunTypes.YARN_SESSION);
 		LOG.info("Finished testfullAlloc()");
 		checkForLogString("There is not enough memory available in the YARN cluster. The TaskManager(s) require 3840MB each. NodeManagers available: [4096, 4096]\n" +
 				"After allocating the JobManager (512MB) and (1/2) TaskManagers, the following NodeManagers are available: [3584, 256]");
@@ -389,15 +378,145 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		LOG.info("Starting perJobYarnCluster()");
 		File exampleJarLocation = YarnTestBase.findFile("..", new ContainsName("-WordCount.jar", "streaming")); // exclude streaming wordcount here.
 		Assert.assertNotNull("Could not find wordcount jar", exampleJarLocation);
-		runWithArgs(new String[] {"run", "-m", "yarn-cluster",
-				"-yj", flinkUberjar.getAbsolutePath(),
-				"-yn", "1",
-				"-yjm", "512",
-				"-ytm", "1024", exampleJarLocation.getAbsolutePath()}, "Job execution switched to status FINISHED.", RunTypes.CLI_FRONTEND);
+		runWithArgs(new String[]{"run", "-m", "yarn-cluster",
+						"-yj", flinkUberjar.getAbsolutePath(),
+						"-yn", "1",
+						"-ys", "2", //test that the job is executed with a DOP of 2
+						"-yjm", "512",
+						"-ytm", "1024", exampleJarLocation.getAbsolutePath()},
+				/* test succeeded after this string */
+				"Job execution switched to status FINISHED.",
+				/* prohibited strings: (we want to see (2/2)) */
+				new String[]{"System.out)(1/1) switched to FINISHED "},
+				RunTypes.CLI_FRONTEND);
 		LOG.info("Finished perJobYarnCluster()");
 	}
 
 	/**
+	 * Test per-job yarn cluster with the parallelism set at the CliFrontend instead of the YARN client.
+	 */
+	@Test
+	public void perJobYarnClusterWithParallelism() {
+		LOG.info("Starting perJobYarnCluster()");
+		File exampleJarLocation = YarnTestBase.findFile("..", new ContainsName("-WordCount.jar", "streaming")); // exclude streaming wordcount here.
+		Assert.assertNotNull("Could not find wordcount jar", exampleJarLocation);
+		runWithArgs(new String[]{"run",
+						"-p", "2", //test that the job is executed with a DOP of 2
+						"-m", "yarn-cluster",
+						"-yj", flinkUberjar.getAbsolutePath(),
+						"-yn", "1",
+						"-yjm", "512",
+						"-ytm", "1024", exampleJarLocation.getAbsolutePath()},
+				/* test succeeded after this string */
+				"Job execution switched to status FINISHED.",
+				/* prohibited strings: (we want to see (2/2)) */
+				new String[]{"System.out)(1/1) switched to FINISHED "},
+				RunTypes.CLI_FRONTEND);
+		LOG.info("Finished perJobYarnCluster()");
+	}
+
+	/**
+	 * Test a fire-and-forget job submission to a YARN cluster.
+	 */
+	@Test(timeout=60000)
+	public void testDetachedPerJobYarnCluster() {
+		LOG.info("Starting testDetachedPerJobYarnCluster()");
+
+		File exampleJarLocation = YarnTestBase.findFile("..", new ContainsName("-WordCount.jar", "streaming")); // exclude streaming wordcount here.
+		Assert.assertNotNull("Could not find wordcount jar", exampleJarLocation);
+
+		YarnClient yc = YarnClient.createYarnClient();
+		yc.init(yarnConfiguration);
+		yc.start();
+
+		Runner runner = startWithArgs(new String[]{"run", "-m", "yarn-cluster", "-yj", flinkUberjar.getAbsolutePath(),
+					"-yn", "1",
+					"-yjm", "512",
+					"-yD", "yarn.heap-cutoff-ratio=0.5", // test if the cutoff is passed correctly
+					"-ytm", "1024",
+					"--yarndetached", exampleJarLocation.getAbsolutePath()},
+			"The Job has been submitted with JobID",
+			RunTypes.CLI_FRONTEND);
+
+		// it should usually be 2, but on slow machines, the number varies
+		Assert.assertTrue("There should be at most 2 containers running", getRunningContainers() <= 2);
+		Assert.assertFalse("The runner should detach.", runner.isAlive());
+		LOG.info("CLI Frontend has returned, so the job is running");
+
+		// find out the application id and wait until it has finished.
+		try {
+			List<ApplicationReport> apps = yc.getApplications(EnumSet.of(YarnApplicationState.RUNNING));
+
+			ApplicationId tmpAppId = null;
+			if (apps.size() == 1) {
+				// Better method to find the right appId. But sometimes the app is shutting down very fast
+				// Only one running
+				tmpAppId = apps.get(0).getApplicationId();
+
+				LOG.info("waiting for the job with appId {} to finish", tmpAppId);
+				// wait until the app has finished
+				while(yc.getApplications(EnumSet.of(YarnApplicationState.RUNNING)).size() == 0) {
+					sleep(500);
+				}
+			} else {
+				// get appId by finding the latest finished appid
+				apps = yc.getApplications();
+				Collections.sort(apps, new Comparator<ApplicationReport>() {
+					@Override
+					public int compare(ApplicationReport o1, ApplicationReport o2) {
+						return o1.getApplicationId().compareTo(o2.getApplicationId())*-1;
+					}
+				});
+				tmpAppId = apps.get(0).getApplicationId();
+				LOG.info("Selected {} as the last appId from {}", tmpAppId, Arrays.toString(apps.toArray()));
+			}
+			final ApplicationId id = tmpAppId;
+			
+			// now it has finished.
+			// check the output.
+			File taskmanagerOut = YarnTestBase.findFile("..", new FilenameFilter() {
+				@Override
+				public boolean accept(File dir, String name) {
+					return name.contains("taskmanager") && name.contains("stdout") && dir.getAbsolutePath().contains(id.toString());
+				}
+			});
+			Assert.assertNotNull("Taskmanager output not found", taskmanagerOut);
+			LOG.info("The job has finished. TaskManager output file found {}", taskmanagerOut.getAbsolutePath());
+			String content = FileUtils.readFileToString(taskmanagerOut);
+			// check for some of the wordcount outputs.
+			Assert.assertTrue("Expected string '(all,2)' not found ", content.contains("(all,2)"));
+			Assert.assertTrue("Expected string '(mind,1)' not found", content.contains("(mind,1)"));
+
+			// check if the heap size for the TaskManager was set correctly
+			File jobmanagerLog = YarnTestBase.findFile("..", new FilenameFilter() {
+				@Override
+				public boolean accept(File dir, String name) {
+					return name.contains("jobmanager-main") && dir.getAbsolutePath().contains(id.toString());
+				}
+			});
+			content = FileUtils.readFileToString(jobmanagerLog);
+			// expecting 512 mb, because TM was started with 1024, we cut off 50% (NOT THE DEFAULT VALUE).
+			Assert.assertTrue("Expected string 'Starting TM with command=$JAVA_HOME/bin/java -Xmx512m' not found in JobManager log",
+					content.contains("Starting TM with command=$JAVA_HOME/bin/java -Xmx512m"));
+
+			// make sure the detached app is really finished.
+			LOG.info("Checking again that app has finished");
+			ApplicationReport rep;
+			do {
+				sleep(500);
+				rep = yc.getApplicationReport(id);
+				LOG.info("Got report {}", rep);
+			} while(rep.getYarnApplicationState() == YarnApplicationState.RUNNING);
+
+		} catch(Throwable t) {
+			LOG.warn("Error while detached yarn session was running", t);
+			Assert.fail();
+		}
+
+		LOG.info("Finished testDetachedPerJobYarnCluster()");
+	}
+
+	/**
 	 * Test the YARN Java API
 	 */
 	@Test
@@ -419,6 +538,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		AbstractFlinkYarnCluster yarnCluster = null;
 		try {
 			yarnCluster = flinkYarnClient.deploy(null);
+			yarnCluster.connectToCluster();
 		} catch (Exception e) {
 			System.err.println("Error while deploying YARN cluster: "+e.getMessage());
 			LOG.warn("Failing test", e);
@@ -453,8 +573,9 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		LOG.info("Finished testJavaAPI()");
 	}
 
-	public boolean ignoreOnTravis() {
-		if(System.getenv("TRAVIS") != null && System.getenv("TRAVIS").equals("true")) {
+
+	public static boolean ignoreOnTravis() {
+		if(isOnTravis()) {
 			// we skip the test until we are able to start a smaller yarn clsuter
 			// right now, the miniyarncluster has the size of the nodemanagers fixed on 4 GBs.
 			LOG.warn("Skipping test on travis for now");

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java
index db1766a..ce78944 100644
--- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YarnTestBase.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.yarn;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.flink.client.CliFrontend;
 import org.apache.flink.client.FlinkYarnSessionCli;
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +37,7 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -83,13 +85,16 @@ public abstract class YarnTestBase {
 	};
 
 	// Temp directory which is deleted after the unit test.
-	private static TemporaryFolder tmp = new TemporaryFolder();
+	@ClassRule
+	public static TemporaryFolder tmp = new TemporaryFolder();
 
 	protected static MiniYARNCluster yarnCluster = null;
 
 	protected static File flinkUberjar;
 
 	protected static final Configuration yarnConfiguration;
+	protected static final String oldHome = System.getProperty("user.home");
+
 	static {
 		yarnConfiguration = new YarnConfiguration();
 		yarnConfiguration.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
@@ -102,7 +107,7 @@ public abstract class YarnTestBase {
 		yarnConfiguration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false);
 		yarnConfiguration.setInt(YarnConfiguration.NM_VCORES, 666); // memory is overwritten in the MiniYARNCluster.
 		// so we have to change the number of cores for testing.
-		yarnConfiguration.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 10000); // 10 seconds expiry (to ensure we properly heartbeat with YARN).
+		yarnConfiguration.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 20000); // 20 seconds expiry (to ensure we properly heartbeat with YARN).
 	}
 
 	// This code is taken from: http://stackoverflow.com/a/7201825/568695
@@ -153,11 +158,15 @@ public abstract class YarnTestBase {
 		}
 	}
 
+	private YarnClient yarnClient = null;
 	@Before
 	public void checkClusterEmpty() throws IOException, YarnException {
-		YarnClient yarnClient = YarnClient.createYarnClient();
-		yarnClient.init(yarnConfiguration);
-		yarnClient.start();
+		if(yarnClient == null) {
+			yarnClient = YarnClient.createYarnClient();
+			yarnClient.init(yarnConfiguration);
+			yarnClient.start();
+		}
+
 		List<ApplicationReport> apps = yarnClient.getApplications();
 		for(ApplicationReport app : apps) {
 			if(app.getYarnApplicationState() != YarnApplicationState.FINISHED
@@ -201,6 +210,7 @@ public abstract class YarnTestBase {
 			return name.startsWith("flink-dist") && name.endsWith(".jar") && dir.toString().contains("/lib");
 		}
 	}
+
 	public static class ContainsName implements FilenameFilter {
 		private String name;
 		private String excludeInPath = null;
@@ -305,8 +315,19 @@ public abstract class YarnTestBase {
 	}
 
 	public static void startYARNWithConfig(Configuration conf) {
-		flinkUberjar = findFile("..", new RootDirFilenameFilter());
-		Assert.assertNotNull(flinkUberjar);
+		// set the home directory to a tmp directory. Flink on YARN is using the home dir to distribute the file
+		File homeDir = null;
+		try {
+			homeDir = tmp.newFolder();
+		} catch (IOException e) {
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+		System.setProperty("user.home", homeDir.getAbsolutePath());
+		String uberjarStartLoc = "..";
+		LOG.info("Trying to locate uberjar in {}", new File(uberjarStartLoc));
+		flinkUberjar = findFile(uberjarStartLoc, new RootDirFilenameFilter());
+		Assert.assertNotNull("Flink uberjar not found", flinkUberjar);
 		String flinkDistRootDir = flinkUberjar.getParentFile().getParent();
 
 		if (!flinkUberjar.exists()) {
@@ -396,7 +417,7 @@ public abstract class YarnTestBase {
 	/**
 	 * The test has been passed once the "terminateAfterString" has been seen.
 	 */
-	protected void runWithArgs(String[] args, String terminateAfterString, RunTypes type) {
+	protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnStrings, RunTypes type) {
 		LOG.info("Running with args {}", Arrays.toString(args));
 
 		outContent = new ByteArrayOutputStream();
@@ -413,9 +434,23 @@ public abstract class YarnTestBase {
 		boolean expectedStringSeen = false;
 		for(int second = 0; second <  START_TIMEOUT_SECONDS; second++) {
 			sleep(1000);
+			String outContentString = outContent.toString();
+			String errContentString = errContent.toString();
+			if(failOnStrings != null) {
+				for(int i = 0; i < failOnStrings.length; i++) {
+					if(outContentString.contains(failOnStrings[i])
+							|| errContentString.contains(failOnStrings[i])) {
+						LOG.warn("Failing test. Output contained illegal string '"+ failOnStrings[i]+"'");
+						sendOutput();
+						// stopping runner.
+						runner.sendStop();
+						Assert.fail("Output contained illegal string '"+ failOnStrings[i]+"'");
+					}
+				}
+			}
 			// check output for correct TaskManager startup.
-			if(outContent.toString().contains(terminateAfterString)
-					|| errContent.toString().contains(terminateAfterString) ) {
+			if(outContentString.contains(terminateAfterString)
+					|| errContentString.contains(terminateAfterString) ) {
 				expectedStringSeen = true;
 				LOG.info("Found expected output in redirected streams");
 				// send "stop" command to command line interface
@@ -502,10 +537,31 @@ public abstract class YarnTestBase {
 	public static void tearDown() {
 		//shutdown YARN cluster
 		if (yarnCluster != null) {
-			LOG.info("shutdown MiniYarn cluster");
+			LOG.info("Shutting down MiniYarn cluster");
 			yarnCluster.stop();
 			yarnCluster = null;
 		}
+		// When we are on travis, we copy the tmp files of JUnit (containing the MiniYARNCluster log files)
+		// to <flinkRoot>/target/flink-yarn-tests-*.
+		// The files from there are picked up by the ./tools/travis_watchdog.sh script
+		// to upload them to Amazon S3.
+		if(isOnTravis()) {
+			File target = new File("../target/"+yarnConfiguration.get(TEST_CLUSTER_NAME_KEY));
+			if(!target.mkdirs()) {
+				LOG.warn("Error creating dirs to {}", target);
+			}
+			File src = tmp.getRoot();
+			LOG.info("copying the final files from {} to {}", src.getAbsolutePath(), target.getAbsolutePath());
+			try {
+				FileUtils.copyDirectoryToDirectory(src, target);
+			} catch (IOException e) {
+				LOG.warn("Error copying the final files from {} to {}: msg: {}", src.getAbsolutePath(), target.getAbsolutePath(), e.getMessage(), e);
+			}
+		}
+	}
+
+	public static boolean isOnTravis() {
+		return System.getenv("TRAVIS") != null && System.getenv("TRAVIS").equals("true");
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn-tests/src/main/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/resources/log4j-test.properties b/flink-yarn-tests/src/main/resources/log4j-test.properties
index 3fee7d6..dc02575 100644
--- a/flink-yarn-tests/src/main/resources/log4j-test.properties
+++ b/flink-yarn-tests/src/main/resources/log4j-test.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 ################################################################################
 
-log4j.rootLogger=FATAL, console
+log4j.rootLogger=INFO, console
 
 # Log all infos in the given file
 log4j.appender.console=org.apache.log4j.ConsoleAppender

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
index 077ed11..9870973 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
@@ -30,6 +30,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.CliFrontend;
 import org.apache.flink.client.FlinkYarnSessionCli;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
@@ -90,6 +92,7 @@ public class FlinkYarnClient extends AbstractFlinkYarnClient {
 	public static final String ENV_CLIENT_SHIP_FILES = "_CLIENT_SHIP_FILES";
 	public static final String ENV_CLIENT_USERNAME = "_CLIENT_USERNAME";
 	public static final String ENV_SLOTS = "_SLOTS";
+	public static final String ENV_DETACHED = "_DETACHED";
 	public static final String ENV_DYNAMIC_PROPERTIES = "_DYNAMIC_PROPERTIES";
 
 
@@ -299,6 +302,16 @@ public class FlinkYarnClient extends AbstractFlinkYarnClient {
 		return false;
 	}
 
+	@Override
+	public void setDetachedMode(boolean detachedMode) {
+		this.detached = detachedMode;
+	}
+
+	@Override
+	public boolean isDetached() {
+		return detached;
+	}
+
 	public AbstractFlinkYarnCluster deploy(final String clusterName) throws Exception {
 
 		UserGroupInformation.setConfiguration(conf);
@@ -320,10 +333,7 @@ public class FlinkYarnClient extends AbstractFlinkYarnClient {
 		}
 	}
 
-	@Override
-	public void setDetachedMode(boolean detachedMode) {
-		this.detached = detachedMode;
-	}
+
 
 	/**
 	 * This method will block until the ApplicationMaster/JobManager have been
@@ -341,6 +351,13 @@ public class FlinkYarnClient extends AbstractFlinkYarnClient {
 		yarnApplication = yarnClient.createApplication();
 		GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse();
 
+		// ------------------ Add dynamic properties to local flinkConfiguraton ------
+
+		List<Tuple2<String, String>> dynProperties = CliFrontend.getDynamicProperties(dynamicPropertiesEncoded);
+		for (Tuple2<String, String> dynProperty : dynProperties) {
+			flinkConfiguration.setString(dynProperty.f0, dynProperty.f1);
+		}
+
 		// ------------------ Check if the specified queue exists --------------
 
 		try {
@@ -466,7 +483,7 @@ public class FlinkYarnClient extends AbstractFlinkYarnClient {
 				.newRecord(ContainerLaunchContext.class);
 
 		String amCommand = "$JAVA_HOME/bin/java"
-					+ " -Xmx"+Utils.calculateHeapSize(jobManagerMemoryMb)+"M " +javaOpts;
+					+ " -Xmx"+Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration)+"M " +javaOpts;
 
 		if(hasLogback || hasLog4j) {
 			amCommand += " -Dlog.file=\""+ApplicationConstants.LOG_DIR_EXPANSION_VAR +"/jobmanager-main.log\"";
@@ -558,6 +575,8 @@ public class FlinkYarnClient extends AbstractFlinkYarnClient {
 		appMasterEnv.put(FlinkYarnClient.ENV_CLIENT_SHIP_FILES, envShipFileList.toString());
 		appMasterEnv.put(FlinkYarnClient.ENV_CLIENT_USERNAME, UserGroupInformation.getCurrentUser().getShortUserName());
 		appMasterEnv.put(FlinkYarnClient.ENV_SLOTS, String.valueOf(slots));
+		appMasterEnv.put(FlinkYarnClient.ENV_DETACHED, String.valueOf(detached));
+
 		if(dynamicPropertiesEncoded != null) {
 			appMasterEnv.put(FlinkYarnClient.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
index 804b2e1..6dd84d6 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
@@ -25,6 +25,7 @@ import static akka.pattern.Patterns.ask;
 import akka.actor.Props;
 import akka.pattern.Patterns;
 import akka.util.Timeout;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.net.NetUtils;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
@@ -82,6 +83,10 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 	private final Timeout akkaTimeout;
 	private final ApplicationId applicationId;
 	private final boolean detached;
+	private final org.apache.flink.configuration.Configuration flinkConfig;
+	private final ApplicationId appId;
+
+	private boolean isConnected = false;
 
 
 	/**
@@ -106,35 +111,47 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 		this.sessionFilesDir = sessionFilesDir;
 		this.applicationId = appId;
 		this.detached = detached;
+		this.flinkConfig = flinkConfig;
+		this.appId = appId;
 
 		// get one application report manually
 		intialAppReport = yarnClient.getApplicationReport(appId);
 		String jobManagerHost = intialAppReport.getHost();
 		int jobManagerPort = intialAppReport.getRpcPort();
 		this.jobManagerAddress = new InetSocketAddress(jobManagerHost, jobManagerPort);
+	}
 
-		if(!detached) {
-			// start actor system
-			LOG.info("Start actor system.");
-			InetAddress ownHostname = NetUtils.resolveAddress(jobManagerAddress); // find name of own public interface, able to connect to the JM
-			actorSystem = AkkaUtils.createActorSystem(flinkConfig,
-					new Some(new Tuple2<String, Integer>(ownHostname.getCanonicalHostName(), 0)));
+	/**
+	 * Connect the FlinkYarnCluster to the ApplicationMaster.
+	 *
+	 * Detached YARN sessions don't need to connect to the ApplicationMaster.
+	 * Detached per job YARN sessions need to connect until the required number of TaskManagers have been started.
+	 * 
+	 * @throws IOException
+	 */
+	public void connectToCluster() throws IOException {
+		if(isConnected) {
+			throw new IllegalStateException("Can not connect to the cluster again");
+		}
 
-			// start application client
-			LOG.info("Start application client.");
+		// start actor system
+		LOG.info("Start actor system.");
+		InetAddress ownHostname = NetUtils.resolveAddress(jobManagerAddress); // find name of own public interface, able to connect to the JM
+		actorSystem = AkkaUtils.createActorSystem(flinkConfig,
+				new Some(new Tuple2<String, Integer>(ownHostname.getCanonicalHostName(), 0)));
 
-			applicationClient = actorSystem.actorOf(Props.create(ApplicationClient.class), "applicationClient");
+		// start application client
+		LOG.info("Start application client.");
 
-			// instruct ApplicationClient to start a periodical status polling
-			applicationClient.tell(new Messages.LocalRegisterClient(this.jobManagerAddress), applicationClient);
+		applicationClient = actorSystem.actorOf(Props.create(ApplicationClient.class, flinkConfig), "applicationClient");
 
+		// instruct ApplicationClient to start a periodical status polling
+		applicationClient.tell(new Messages.LocalRegisterClient(this.jobManagerAddress), applicationClient);
 
-			// add hook to ensure proper shutdown
-			Runtime.getRuntime().addShutdownHook(clientShutdownHook);
 
-			actorRunner = new Thread(new Runnable() {
-				@Override
-				public void run() {
+		actorRunner = new Thread(new Runnable() {
+			@Override
+			public void run() {
 				// blocks until ApplicationMaster has been stopped
 				actorSystem.awaitTermination();
 
@@ -157,19 +174,66 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 				} catch (Exception e) {
 					LOG.warn("Error while getting final application report", e);
 				}
-				}
-			});
-			actorRunner.setDaemon(true);
-			actorRunner.start();
+			}
+		});
+		actorRunner.setDaemon(true);
+		actorRunner.start();
 
-			pollingRunner = new PollingThread(yarnClient, appId);
-			pollingRunner.setDaemon(true);
-			pollingRunner.start();
+		pollingRunner = new PollingThread(yarnClient, appId);
+		pollingRunner.setDaemon(true);
+		pollingRunner.start();
+
+		Runtime.getRuntime().addShutdownHook(clientShutdownHook);
+
+		isConnected = true;
+	}
+
+	@Override
+	public void disconnect() {
+		if(!isConnected) {
+			throw new IllegalStateException("Can not disconnect from an unconnected cluster.");
+		}
+		LOG.info("Disconnecting FlinkYarnCluster from ApplicationMaster");
+
+		if(!Runtime.getRuntime().removeShutdownHook(clientShutdownHook)) {
+			LOG.warn("Error while removing the shutdown hook. The YARN session might be killed unintentionally");
+		}
+		// tell the actor to shut down.
+		applicationClient.tell(Messages.getLocalUnregisterClient(), applicationClient);
+
+		try {
+			actorRunner.join(1000); // wait for 1 second
+		} catch (InterruptedException e) {
+			LOG.warn("Shutdown of the actor runner was interrupted", e);
+			Thread.currentThread().interrupt();
 		}
+		try {
+			pollingRunner.stopRunner();
+			pollingRunner.join(1000);
+		} catch(InterruptedException e) {
+			LOG.warn("Shutdown of the polling runner was interrupted", e);
+			Thread.currentThread().interrupt();
+		}
+		isConnected = false;
 	}
 
+
 	// -------------------------- Interaction with the cluster ------------------------
 
+	/**
+	 * This call blocks until the message has been recevied.
+	 * @param jobID
+	 */
+	@Override
+	public void stopAfterJob(JobID jobID) {
+		Future<Object> messageReceived = ask(applicationClient, new Messages.StopAMAfterJob(jobID), akkaTimeout);
+		try {
+			Await.result(messageReceived, akkaDuration);
+		} catch (Exception e) {
+			throw new RuntimeException("Unable to tell application master to stop once the specified job has been finised", e);
+		}
+	}
+
 	@Override
 	public InetSocketAddress getJobManagerAddress() {
 		return jobManagerAddress;
@@ -190,14 +254,18 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 		return applicationId.toString();
 	}
 
+	@Override
+	public boolean isDetached() {
+		return this.detached;
+	}
+
 	/**
 	 * This method is only available if the cluster hasn't been started in detached mode.
 	 */
 	@Override
 	public FlinkYarnClusterStatus getClusterStatus() {
-		if(detached) {
-			throw new IllegalArgumentException("The cluster has been started in detached mode." +
-					"Can not request cluster status");
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
 		}
 		if(hasBeenStopped()) {
 			throw new RuntimeException("The FlinkYarnCluster has alread been stopped");
@@ -220,9 +288,8 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 
 	@Override
 	public boolean hasFailed() {
-		if(detached) {
-			throw new IllegalArgumentException("The cluster has been started in detached mode." +
-					"Can not request cluster status");
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
 		}
 		if(pollingRunner == null) {
 			LOG.warn("FlinkYarnCluster.hasFailed() has been called on an uninitialized cluster." +
@@ -248,10 +315,10 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 
 	@Override
 	public String getDiagnostics() {
-		if(detached) {
-			throw new IllegalArgumentException("The cluster has been started in detached mode." +
-					"Can not request cluster status");
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
 		}
+
 		if (!hasFailed()) {
 			LOG.warn("getDiagnostics() called for cluster which is not in failed state");
 		}
@@ -266,10 +333,10 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 
 	@Override
 	public List<String> getNewMessages() {
-		if(detached) {
-			throw new IllegalArgumentException("The cluster has been started in detached mode." +
-					"Can not request cluster status");
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
 		}
+
 		if(hasBeenStopped()) {
 			throw new RuntimeException("The FlinkYarnCluster has already been stopped");
 		}
@@ -320,10 +387,10 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 	}
 
 	private void shutdownInternal(boolean removeShutdownHook) {
-		if(detached) {
-			throw new IllegalArgumentException("The cluster has been started in detached mode." +
-					"Can not control a detached cluster");
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
 		}
+
 		if(hasBeenShutDown.getAndSet(true)) {
 			return;
 		}
@@ -389,6 +456,7 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
 	public class ClientShutdownHook extends Thread {
 		@Override
 		public void run() {
+			LOG.info("Shutting down FlinkYarnCluster from the client shutdown hook");
 			shutdownInternal(false);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
index 3d2c2fa..5a9e65b 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -64,9 +63,9 @@ public class Utils {
 	 * more than 500MB (the current HEAP_LIMIT_CAP), we'll just subtract 500 MB.
 	 * 
 	 */
-	public static int calculateHeapSize(int memory) {
-		float memoryCutoffRatio = GlobalConfiguration.getFloat(ConfigConstants.YARN_HEAP_CUTOFF_RATIO, DEFAULT_YARN_HEAP_CUTOFF_RATIO);
-		int heapLimitCap = GlobalConfiguration.getInteger(ConfigConstants.YARN_HEAP_LIMIT_CAP, DEFAULT_HEAP_LIMIT_CAP);
+	public static int calculateHeapSize(int memory, org.apache.flink.configuration.Configuration conf) {
+		float memoryCutoffRatio = conf.getFloat(ConfigConstants.YARN_HEAP_CUTOFF_RATIO, DEFAULT_YARN_HEAP_CUTOFF_RATIO);
+		int heapLimitCap = conf.getInteger(ConfigConstants.YARN_HEAP_LIMIT_CAP, DEFAULT_HEAP_LIMIT_CAP);
 
 		int heapLimit = (int)((float)memory * memoryCutoffRatio);
 		if( (memory - heapLimit) > heapLimitCap) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
index 731f8e2..b400bb9 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
@@ -22,7 +22,7 @@ import java.net.InetSocketAddress
 
 import akka.actor._
 import akka.pattern.ask
-import org.apache.flink.configuration.GlobalConfiguration
+import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.jobmanager.JobManager
@@ -34,7 +34,8 @@ import scala.concurrent.duration._
 import scala.language.postfixOps
 import scala.util.{Failure, Success}
 
-class ApplicationClient extends Actor with ActorLogMessages with ActorLogging {
+class ApplicationClient(flinkConfig: Configuration) extends Actor
+  with ActorLogMessages with ActorLogging {
   import context._
 
   val INITIAL_POLLING_DELAY = 0 seconds
@@ -52,7 +53,7 @@ class ApplicationClient extends Actor with ActorLogMessages with ActorLogging {
   override def preStart(): Unit = {
     super.preStart()
 
-    timeout = AkkaUtils.getTimeout(GlobalConfiguration.getConfiguration())
+    timeout = AkkaUtils.getTimeout(flinkConfig)
   }
 
   override def postStop(): Unit = {
@@ -101,6 +102,14 @@ class ApplicationClient extends Actor with ActorLogMessages with ActorLogging {
       pollingTimer = Some(context.system.scheduler.schedule(INITIAL_POLLING_DELAY,
         WAIT_FOR_YARN_INTERVAL, jm, PollYarnClusterStatus))
 
+    case LocalUnregisterClient =>
+      // unregister client from AM
+      yarnJobManager foreach {
+        _ ! UnregisterClient
+      }
+      // poison ourselves
+      self ! PoisonPill
+
     case msg: StopYarnSession =>
       log.info("Sending StopYarnSession request to ApplicationMaster.")
       stopMessageReceiver = Some(sender)
@@ -126,6 +135,11 @@ class ApplicationClient extends Actor with ActorLogMessages with ActorLogging {
     case LocalGetYarnClusterStatus =>
       sender() ! latestClusterStatus
 
+      // Forward message to Application Master
+    case msg: StopAMAfterJob =>
+      yarnJobManager foreach {
+        _ forward msg
+      }
 
     // -----------------  handle messages from the cluster -------------------
     // receive remote messages

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
index f88685c..4285da7 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
@@ -23,9 +23,11 @@ import java.nio.ByteBuffer
 import java.util.Collections
 
 import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
 import org.apache.flink.configuration.ConfigConstants
 import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.messages.JobManagerMessages.{CurrentJobStatus, JobNotFound, RequestJobStatus}
 import org.apache.flink.runtime.messages.Messages.Acknowledge
 import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus
 import org.apache.flink.yarn.Messages._
@@ -58,13 +60,19 @@ trait ApplicationMasterActor extends ActorLogMessages {
   val FAST_YARN_HEARTBEAT_DELAY: FiniteDuration = 500 milliseconds
   val DEFAULT_YARN_HEARTBEAT_DELAY: FiniteDuration = 5 seconds
   val YARN_HEARTBEAT_DELAY: FiniteDuration =
-    if(configuration.getString(ConfigConstants.YARN_HEARTBEAT_DELAY_SECONDS, null) == null) {
+    if(flinkConfiguration.getString(ConfigConstants.YARN_HEARTBEAT_DELAY_SECONDS, null) == null) {
       DEFAULT_YARN_HEARTBEAT_DELAY
     } else {
       FiniteDuration(
-        configuration.getInteger(ConfigConstants.YARN_HEARTBEAT_DELAY_SECONDS, 5), SECONDS)
+        flinkConfiguration.getInteger(ConfigConstants.YARN_HEARTBEAT_DELAY_SECONDS, 5), SECONDS)
     }
 
+  private def env = System.getenv()
+
+  // indicates if this AM has been started in a detached mode.
+  val detached = java.lang.Boolean.valueOf(env.get(FlinkYarnClient.ENV_DETACHED))
+  var stopWhenJobFinished: JobID = null
+
   var rmClientOption: Option[AMRMClient[ContainerRequest]] = None
   var nmClientOption: Option[NMClient] = None
   var messageListener:Option[ActorRef] = None
@@ -129,6 +137,16 @@ trait ApplicationMasterActor extends ActorLogMessages {
       messageListener = Some(client)
       sender ! Acknowledge
 
+    case UnregisterClient =>
+      messageListener = None
+
+    case msg: StopAMAfterJob =>
+      val jobId = msg.jobId
+      log.info("ApplicatonMaster will shut down YARN session when job {} has finished", jobId)
+      stopWhenJobFinished = jobId
+      sender() ! Acknowledge
+
+
     case PollYarnClusterStatus =>
       sender() ! new FlinkYarnClusterStatus(instanceManager.getNumberOfRegisteredTaskManagers,
         instanceManager.getTotalNumberOfSlots)
@@ -136,7 +154,36 @@ trait ApplicationMasterActor extends ActorLogMessages {
     case StartYarnSession(conf, actorSystemPort, webServerPort) =>
       startYarnSession(conf, actorSystemPort, webServerPort)
 
+    case jnf: JobNotFound =>
+      LOG.warn("Job with ID {} not found in JobManager", jnf.jobID)
+      if(stopWhenJobFinished == null) {
+        LOG.warn("The ApplicationMaster didn't expect to receive this message")
+      }
+
+    case jobStatus: CurrentJobStatus =>
+      if(stopWhenJobFinished == null) {
+        LOG.warn("Received job status {} which wasn't requested", jobStatus)
+      } else {
+        if(stopWhenJobFinished != jobStatus.jobID) {
+          LOG.warn(s"Received job status for job ${jobStatus.jobID} but expected status for " +
+            s"job $stopWhenJobFinished")
+        } else {
+          if(jobStatus.status.isTerminalState) {
+            LOG.info(s"Job with ID ${jobStatus.jobID} is in terminal state ${jobStatus.status}. " +
+              s"Shutting down YARN session")
+            self ! StopYarnSession(FinalApplicationStatus.SUCCEEDED,
+              s"The monitored job with ID ${jobStatus.jobID} has finished.")
+          } else {
+            LOG.debug(s"Monitored job with ID ${jobStatus.jobID} is in state ${jobStatus.status}")
+          }
+        }
+      }
+
     case HeartbeatWithYarn =>
+      // piggyback on the YARN heartbeat to check if the job has finished
+      if(stopWhenJobFinished != null) {
+        self ! RequestJobStatus(stopWhenJobFinished)
+      }
       rmClientOption match {
         case Some(rmClient) =>
           log.debug("Send heartbeat to YARN")
@@ -222,7 +269,7 @@ trait ApplicationMasterActor extends ActorLogMessages {
                             nmClient.startContainer(container, ctx)
                             runningContainers += 1
                             missingContainers -= 1
-                            val message = s"Launching container ${containersLaunched} " +
+                            val message = s"Launching container $containersLaunched " +
                               s"(${container.getId} on host ${container.getNodeId.getHost})."
                             log.info(message)
                             containersLaunched += 1
@@ -253,7 +300,7 @@ trait ApplicationMasterActor extends ActorLogMessages {
             // if there are still containers missing, request them from YARN
             val toAllocateFromYarn = Math.max(missingContainers - numPendingRequests, 0)
             if(toAllocateFromYarn > 0) {
-              val reallocate = configuration
+              val reallocate = flinkConfiguration
                 .getBoolean(ConfigConstants.YARN_REALLOCATE_FAILED_CONTAINERS, true)
               log.info(s"There are $missingContainers containers missing." +
                 s" $numPendingRequests are already requested. " +
@@ -282,7 +329,8 @@ trait ApplicationMasterActor extends ActorLogMessages {
             allocatedContainersList.clear()
           }
 
-          if(failedContainers >= maxFailedContainers) {
+          // maxFailedContainers == -1 is infinite number of retries.
+          if(maxFailedContainers != -1 && failedContainers >= maxFailedContainers) {
             val msg = s"Stopping YARN session because the number of failed " +
               s"containers ($failedContainers) exceeded the maximum failed container " +
               s"count ($maxFailedContainers). This number is controlled by " +
@@ -323,7 +371,7 @@ trait ApplicationMasterActor extends ActorLogMessages {
     Try {
       log.info("Start yarn session.")
       memoryPerTaskManager = env.get(FlinkYarnClient.ENV_TM_MEMORY).toInt
-      val heapLimit = Utils.calculateHeapSize(memoryPerTaskManager)
+      val heapLimit = Utils.calculateHeapSize(memoryPerTaskManager, flinkConfiguration)
 
       val applicationMasterHost = env.get(Environment.NM_HOST.key)
       require(applicationMasterHost != null, s"Application master (${Environment.NM_HOST} not set.")
@@ -339,8 +387,8 @@ trait ApplicationMasterActor extends ActorLogMessages {
       }
 
       numTaskManager = env.get(FlinkYarnClient.ENV_TM_COUNT).toInt
-      maxFailedContainers = configuration.getInteger(ConfigConstants.YARN_MAX_FAILED_CONTAINERS,
-        numTaskManager)
+      maxFailedContainers = flinkConfiguration.
+        getInteger(ConfigConstants.YARN_MAX_FAILED_CONTAINERS, numTaskManager)
       log.info("Requesting {} TaskManagers. Tolerating {} failed TaskManagers",
         numTaskManager, maxFailedContainers)
 
@@ -429,7 +477,7 @@ trait ApplicationMasterActor extends ActorLogMessages {
   private def tryToReturnContainers(returnRequest: mutable.Set[PreemptionContainer]): Unit = {
     for(requestedBackContainers <- returnRequest) {
       allocatedContainersList = allocatedContainersList.dropWhile( container => {
-        val result = requestedBackContainers.equals(container)
+        val result = requestedBackContainers.getId.equals(container.getId)
         if(result) {
           log.info("Returning container {} back to ResourceManager.", container)
         }
@@ -457,7 +505,7 @@ trait ApplicationMasterActor extends ActorLogMessages {
     log.info("Create container launch context.")
     val ctx = Records.newRecord(classOf[ContainerLaunchContext])
 
-    val javaOpts = configuration.getString(ConfigConstants.FLINK_JVM_OPTIONS, "")
+    val javaOpts = flinkConfiguration.getString(ConfigConstants.FLINK_JVM_OPTIONS, "")
     val tmCommand = new StringBuilder(s"$$JAVA_HOME/bin/java -Xmx${heapLimit}m $javaOpts")
 
     if (hasLogback || hasLog4j) {
@@ -504,6 +552,4 @@ trait ApplicationMasterActor extends ActorLogMessages {
 
     ctx
   }
-
-  private def env = System.getenv()
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
index 3b50cc4..99d0345 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala
@@ -22,6 +22,7 @@ import java.net.InetSocketAddress
 import java.util.Date
 
 import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
 
@@ -30,6 +31,7 @@ object Messages {
   case class YarnMessage(message: String, date: Date = new Date())
   case class ApplicationMasterStatus(numTaskManagers: Int, numSlots: Int)
   case class RegisterClient(client: ActorRef)
+  case object UnregisterClient
 
   case class StopYarnSession(status: FinalApplicationStatus, diagnostics: String)
 
@@ -46,12 +48,20 @@ object Messages {
                                     // the response
   case object CheckForUserCommand
 
+  case class StopAMAfterJob(jobId:JobID) // tell the AM to monitor the job and stop once it has
+    // finished.
+
   // Client-local messages
   case class LocalRegisterClient(jobManagerAddress: InetSocketAddress)
+  case object LocalUnregisterClient
   case object LocalGetYarnMessage // request new message
   case object LocalGetYarnClusterStatus // request the latest cluster status
 
   def getLocalGetYarnMessage(): AnyRef = {
     LocalGetYarnMessage
   }
+
+  def getLocalUnregisterClient(): AnyRef = {
+    LocalUnregisterClient
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java
index 39a9c02..546e732 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java
@@ -17,6 +17,8 @@
  */
 package org.apache.flink.yarn;
 
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -24,9 +26,19 @@ public class UtilsTests {
 
 	@Test
 	public void testHeapCutoff() {
-
+		Configuration conf = new Configuration();
 		// ASSUMES DEFAULT Configuration values.
-		Assert.assertEquals(800, Utils.calculateHeapSize(1000) );
-		Assert.assertEquals(9300, Utils.calculateHeapSize(10000) );
+		Assert.assertEquals(800, Utils.calculateHeapSize(1000, conf) );
+		Assert.assertEquals(9300, Utils.calculateHeapSize(10000, conf) );
+
+		// test different configuration
+		Assert.assertEquals(3300, Utils.calculateHeapSize(4000, conf) );
+
+		conf.setString(ConfigConstants.YARN_HEAP_LIMIT_CAP, "1000");
+		conf.setString(ConfigConstants.YARN_HEAP_CUTOFF_RATIO, "0.3");
+		Assert.assertEquals(3000, Utils.calculateHeapSize(4000, conf));
+
+		conf.setString(ConfigConstants.YARN_HEAP_LIMIT_CAP, "6000000");
+		Assert.assertEquals(1200, Utils.calculateHeapSize(4000, conf));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5e87195..7e556ea 100644
--- a/pom.xml
+++ b/pom.xml
@@ -82,7 +82,7 @@ under the License.
 		<log4j.configuration>log4j-test.properties</log4j.configuration>
 		<slf4j.version>1.7.7</slf4j.version>
 		<guava.version>18.0</guava.version>
-		<akka.version>2.3.9</akka.version>
+		<akka.version>2.3.7</akka.version>
 		<scala.macros.version>2.0.1</scala.macros.version>
 		<kryoserialization.version>0.3.2</kryoserialization.version>
 		<protobuf.version>2.5.0</protobuf.version>


[2/4] flink git commit: [FLINK-1771] Add support for submitting single jobs to a detached YARN session

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
index e3aa2f0..98ccfcf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 public abstract class InternalExecutionVertexProfilingData implements InternalProfilingData {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
index 8819fc4..92ce916 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 public class InternalExecutionVertexThreadProfilingData extends InternalExecutionVertexProfilingData {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java
index 5449869..7de9252 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceProfilingEvent.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * Instance profiling events are a special subclass of profiling events. They contain profiling information about the

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java
index ec3ab30..4bb052d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/InstanceSummaryProfilingEvent.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.profiling.types;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * Instance summary profiling events summarize the profiling events of all instances involved in computing a job.

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java
index a4d0634..13a9a99 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ProfilingEvent.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.event.job.AbstractEvent;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java
index 4e0faf7..e07f144 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/SingleInstanceProfilingEvent.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.types.StringValue;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java
index d878906..08b932a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/ThreadProfilingEvent.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java
index 148d16d..fc18758 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/types/VertexProfilingEvent.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index fa2413b..3d1419a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.messages.ExecutionGraphMessages;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
index bda7c8b..a56a7e5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
@@ -25,7 +25,7 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.util.InstantiationUtil;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
index 16ca090..6e446de 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
@@ -24,7 +24,7 @@ import akka.pattern.Patterns;
 import akka.util.Timeout;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.messages.JobManagerMessages;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java
index e8b730d..039e926 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java
@@ -21,35 +21,115 @@ import org.apache.hadoop.fs.Path;
 import java.io.File;
 import java.util.List;
 
+/**
+ * Abstract interface for an implementation of a Flink on YARN client to deploy.
+ *
+ * The Client describes the properties of the YARN application to create.
+ */
 public abstract class AbstractFlinkYarnClient {
 
 	// ---- Setter for YARN Cluster properties ----- //
+
+	/**
+	 * @param memoryMB The amount of memory for the JobManager (in MB)
+	 */
 	public abstract void setJobManagerMemory(int memoryMB);
+
+	/**
+	 * @param memoryMB The memory per TaskManager (in MB)
+	 */
 	public abstract void setTaskManagerMemory(int memoryMB);
 
+	/**
+	 * Flink configuration
+	 */
 	public abstract void setFlinkConfigurationObject(org.apache.flink.configuration.Configuration conf);
 
+	/**
+	 *
+	 * @param slots The number of TaskManager slots per TaskManager.
+	 */
 	public abstract void setTaskManagerSlots(int slots);
+
+	/**
+	 * @return the number of TaskManager processing slots per TaskManager.
+	 */
 	public abstract int getTaskManagerSlots();
+
+	/**
+	 * @param queue Name of the YARN queue
+	 */
 	public abstract void setQueue(String queue);
+
+	/**
+	 *
+	 * @param localJarPath Local Path to the Flink uberjar
+	 */
 	public abstract void setLocalJarPath(Path localJarPath);
+
+	/**
+	 *
+	 * @param confPath local path to the Flink configuration file
+	 */
 	public abstract void setConfigurationFilePath(Path confPath);
+
+	/**
+	 *
+	 * @param logConfPath local path to the flink logging configuration
+	 */
 	public abstract void setFlinkLoggingConfigurationPath(Path logConfPath);
 	public abstract Path getFlinkLoggingConfigurationPath();
+
+	/**
+	 *
+	 * @param tmCount number of TaskManagers to start
+	 */
 	public abstract void setTaskManagerCount(int tmCount);
 	public abstract int getTaskManagerCount();
+
+	/**
+	 * @param confDirPath Path to config directory.
+	 */
 	public abstract void setConfigurationDirectory(String confDirPath);
-	// List of files to transfer to the YARN containers.
+
+	/**
+	 * List of files to transfer to the YARN containers.
+	 */
 	public abstract void setShipFiles(List<File> shipFiles);
+
+	/**
+	 *
+	 * @param dynamicPropertiesEncoded Encoded String of the dynamic properties (-D configuration values of the Flink configuration)
+	 */
 	public abstract void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded);
 	public abstract String getDynamicPropertiesEncoded();
 
-	// ---- Operations on the YARN cluster ----- //
+	// --------------------------------------- Operations on the YARN cluster ----- //
+
+	/**
+	 * Returns a String containing details about the cluster (NodeManagers, available memory, ...)
+	 *
+	 */
 	public abstract String getClusterDescription() throws Exception;
 
+	/**
+	 * Trigger the deployment to YARN.
+	 *
+	 * @param clusterName Name to be shown in the YARN resource manager overview.
+	 */
 	public abstract AbstractFlinkYarnCluster deploy(String clusterName) throws Exception;
 
+	/**
+	 * @param detachedMode If true, the Flink YARN client is non-blocking. That means it returns
+	 *                        once Flink has been started successfully on YARN.
+	 */
 	public abstract void setDetachedMode(boolean detachedMode);
 
+	public abstract boolean isDetached();
+
+	/**
+	 * @return The string representation of the Path to the YARN session files. This is a temporary
+	 * directory in HDFS that contains the jar files and configuration which is shipped to all the containers.
+	 */
 	public abstract String getSessionFilesDir();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
index 4b4bd2d..398709e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.runtime.yarn;
 
+import org.apache.flink.api.common.JobID;
+
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.List;
 
@@ -43,4 +46,30 @@ public abstract class AbstractFlinkYarnCluster {
 	public abstract List<String> getNewMessages();
 
 	public abstract String getApplicationId();
+
+	public abstract boolean isDetached();
+
+	/**
+	 * Connect the FlinkYarnCluster to the ApplicationMaster.
+	 *
+	 * Detached YARN sessions don't need to connect to the ApplicationMaster.
+	 * Detached per job YARN sessions need to connect until the required number of TaskManagers have been started.
+	 *
+	 * @throws IOException
+	 */
+	public abstract void connectToCluster() throws IOException;
+
+	/**
+	 * Disconnect from the ApplicationMaster without stopping the session
+	 * (therefore, use the {@see shutdown()} method.
+	 */
+	public abstract void disconnect();
+
+	/**
+	 * Tells the ApplicationMaster to monitor the status of JobId and stop itself once the specified
+	 * job has finished.
+	 *
+	 * @param jobID
+	 */
+	public abstract void stopAfterJob(JobID jobID);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala
index f1c6450..d153dcc 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala
@@ -88,8 +88,8 @@ ActorLogging {
 
     case Success(_) =>
 
-    case JobResultSuccess(_, duration, accumulatorResults) =>
-      jobSubmitter ! new JobExecutionResult(duration, accumulatorResults)
+    case JobResultSuccess(jobId, duration, accumulatorResults) =>
+      jobSubmitter ! new JobExecutionResult(jobId, duration, accumulatorResults)
       self ! PoisonPill
 
     case msg =>

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index 9aa476d..4b0a55b 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -22,7 +22,7 @@ import java.io.{IOException, File}
 import java.net.InetSocketAddress
 
 import akka.actor.Status.{Success, Failure}
-import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.{JobID, ExecutionConfig}
 import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration, Configuration}
 import org.apache.flink.core.io.InputSplitAssigner
 import org.apache.flink.runtime.blob.BlobServer
@@ -41,7 +41,7 @@ import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.instance.InstanceManager
-import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobID}
+import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus}
 import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager
 import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
 import org.apache.flink.runtime.messages.JobManagerMessages._
@@ -86,7 +86,7 @@ import scala.collection.JavaConverters._
  * - [[JobStatusChanged]] indicates that the status of job (RUNNING, CANCELING, FINISHED, etc.) has
  * changed. This message is sent by the ExecutionGraph.
  */
-class JobManager(val configuration: Configuration,
+class JobManager(val flinkConfiguration: Configuration,
                  val instanceManager: InstanceManager,
                  val scheduler: FlinkScheduler,
                  val libraryCacheManager: BlobLibraryCacheManager,

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
index 0c3384c..cb7bfec 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala
@@ -19,9 +19,9 @@
 package org.apache.flink.runtime.jobmanager
 
 import akka.actor.{ActorLogging, Actor}
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.executiongraph.ExecutionGraph
-import org.apache.flink.runtime.jobgraph.JobID
 import org.apache.flink.runtime.messages.ArchiveMessages._
 import org.apache.flink.runtime.messages.JobManagerMessages._
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/StreamCheckpointCoordinator.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/StreamCheckpointCoordinator.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/StreamCheckpointCoordinator.scala
index b6b2cc1..d9a3421 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/StreamCheckpointCoordinator.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/StreamCheckpointCoordinator.scala
@@ -21,11 +21,12 @@ package org.apache.flink.runtime.jobmanager
 import java.lang.Long
 
 import akka.actor._
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.execution.ExecutionState.RUNNING
 import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph, ExecutionVertex}
 import org.apache.flink.runtime.jobgraph.JobStatus._
-import org.apache.flink.runtime.jobgraph.{JobID, JobVertexID}
+import org.apache.flink.runtime.jobgraph.JobVertexID
 import org.apache.flink.runtime.state.StateHandle
 
 import scala.collection.JavaConversions._

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala
index 704bf86..b4ed2cc 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala
@@ -18,8 +18,8 @@
 
 package org.apache.flink.runtime.messages
 
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.executiongraph.ExecutionGraph
-import org.apache.flink.runtime.jobgraph.JobID
 
 /**
  * This object contains the archive specific messages.

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
index d413d13..6785c31 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala
@@ -21,9 +21,10 @@ package org.apache.flink.runtime.messages
 import java.text.SimpleDateFormat
 import java.util.Date
 
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.execution.ExecutionState
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
-import org.apache.flink.runtime.jobgraph.{JobStatus, JobVertexID, JobID}
+import org.apache.flink.runtime.jobgraph.{JobStatus, JobVertexID}
 
 /**
  * This object contains the execution graph specific messages.

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
index 17e9138..dab4671 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala
@@ -18,12 +18,13 @@
 
 package org.apache.flink.runtime.messages
 
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.accumulators.AccumulatorEvent
 import org.apache.flink.runtime.client.JobStatusMessage
 import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph}
 import org.apache.flink.runtime.instance.{InstanceID, Instance}
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID
-import org.apache.flink.runtime.jobgraph.{JobGraph, JobID, JobStatus, JobVertexID}
+import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID}
 import org.apache.flink.runtime.taskmanager.TaskExecutionState
 
 import scala.collection.JavaConverters._

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
index 0c9bf9b..cb79fbc 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
@@ -193,6 +193,10 @@ class LocalFlinkMiniCluster(userConfiguration: Configuration, singleActorSystem:
     }
   }
 
+  def getConfiguration: Configuration = {
+    this.userConfiguration
+  }
+
   def getDefaultConfig: Configuration = {
     val config: Configuration = new Configuration()
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java
index 2254d7c..db74bbd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java
@@ -31,7 +31,7 @@ import java.net.InetSocketAddress;
 import java.security.MessageDigest;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
index adb3bfc..8613b4e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.blob;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.Test;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
index 1f8d29b..c98e575 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.blob;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.Test;
 
 import java.io.ByteArrayInputStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
index 5742fea..50b154e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
@@ -28,7 +28,7 @@ import java.util.List;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.operators.RegularPactTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
index 239e356..d024135 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.blob.BlobClient;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.blob.BlobServer;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.Test;
 
 import static org.junit.Assert.*;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
index e0852c6..894a7a9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
@@ -49,7 +49,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index c979c42..8c4879f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -46,7 +46,7 @@ import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.operators.RegularPactTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
index d136d6f..5dd10c0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
@@ -39,7 +39,7 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java
index 2d1d9d5..1e78ac5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java
@@ -32,7 +32,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
index 168ea90..e199353 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
@@ -39,7 +39,7 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.messages.TaskManagerMessages;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
index f5089c2..5713c10 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.SimpleSlot;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.messages.TaskManagerMessages.TaskOperationResult;
 import org.apache.flink.runtime.testingUtils.TestingUtils;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
index 66ef4ca..e24a2b4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
@@ -32,7 +32,7 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.SimpleSlot;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
index ddd7282..f72d105 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
@@ -33,7 +33,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 
 public class PointwisePatternTest {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java
index 4f82dea..17d78d2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java
@@ -28,7 +28,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
index df28f27..8623f75 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java
@@ -26,7 +26,7 @@ import org.junit.Assert;
 
 import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry;
 import org.apache.flink.core.fs.local.LocalFileSystem;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
index 47f7a2b..c0ed629 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
@@ -24,7 +24,7 @@ import java.lang.reflect.Method;
 import java.net.InetAddress;
 
 import akka.actor.ActorRef;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java
index b16bf4b..f6c0c9f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java
@@ -25,7 +25,7 @@ import java.net.InetAddress;
 
 import akka.actor.ActorRef;
 import org.apache.flink.runtime.executiongraph.Execution;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.Test;
 import org.mockito.Matchers;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
index cc90c44..32fe1a7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import com.google.common.collect.Lists;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.TaskEventDispatcher;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
@@ -34,7 +35,6 @@ import org.apache.flink.runtime.io.network.util.TestPartitionProducer;
 import org.apache.flink.runtime.io.network.util.TestProducerSource;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.apache.flink.runtime.jobgraph.JobID;
 import org.junit.Test;
 
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobIdTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobIdTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobIdTest.java
index 0405f50..083a1a9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobIdTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobIdTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.jobgraph;
 
+import org.apache.flink.api.common.JobID;
 import org.junit.Test;
 
 import java.nio.ByteBuffer;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
index 006bd27..776184c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
@@ -35,7 +35,7 @@ import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
index fcb638f..42a3702 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
@@ -29,7 +29,7 @@ import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 public class SharedSlotsTest {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
index 23a5c94..d678531 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
@@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.flink.runtime.instance.SimpleSlot;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.junit.Test;
 
 public class SlotAllocationFutureTest {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
index d5c9fbb..7cbb59b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
@@ -35,7 +35,7 @@ import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java
index d7bdda3..c3c75ce 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/types/ProfilingTypesTest.java
@@ -23,11 +23,8 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.profiling.types.InstanceSummaryProfilingEvent;
-import org.apache.flink.runtime.profiling.types.SingleInstanceProfilingEvent;
-import org.apache.flink.runtime.profiling.types.ThreadProfilingEvent;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.testutils.ManagementTestUtils;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
index e9e17ca..6939527 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index 8784c14..640ccc3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -41,7 +41,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.Tasks;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index 68b0e6f..689b22d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -30,7 +30,7 @@ import org.apache.flink.runtime.execution.RuntimeEnvironment;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.MockNetworkEnvironment;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
index 775c0cb..a36ded9 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala
@@ -20,9 +20,10 @@ package org.apache.flink.runtime.executiongraph
 
 import akka.actor.{Props, ActorSystem}
 import akka.testkit.TestKit
+import org.apache.flink.api.common.JobID
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.akka.AkkaUtils
-import org.apache.flink.runtime.jobgraph.{JobStatus, JobID, JobGraph, AbstractJobVertex}
+import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, AbstractJobVertex}
 import org.apache.flink.runtime.jobmanager.Tasks
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
 import org.apache.flink.runtime.testingUtils.TestingUtils

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
index 69b0588..1a36112 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala
@@ -20,11 +20,12 @@ package org.apache.flink.runtime.executiongraph
 
 import akka.actor.{Props, ActorSystem}
 import akka.testkit.TestKit
+import org.apache.flink.api.common.JobID
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils
 .SimpleAcknowledgingTaskManager
-import org.apache.flink.runtime.jobgraph.{JobStatus, JobID, JobGraph, AbstractJobVertex}
+import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, AbstractJobVertex}
 import org.apache.flink.runtime.jobmanager.Tasks
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
 import org.apache.flink.runtime.testingUtils.TestingUtils

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
index 89cbfe6..89e1d72 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
@@ -20,9 +20,10 @@ package org.apache.flink.runtime.testingUtils
 
 import akka.actor.{Cancellable, Terminated, ActorRef, Props}
 import akka.pattern.{ask, pipe}
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.ActorLogMessages
 import org.apache.flink.runtime.execution.ExecutionState
-import org.apache.flink.runtime.jobgraph.{JobStatus, JobID}
+import org.apache.flink.runtime.jobgraph.JobStatus
 import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist}
 import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
 import org.apache.flink.runtime.messages.Messages.Disconnect

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
index a5a577b..f810749 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
@@ -19,8 +19,9 @@
 package org.apache.flink.runtime.testingUtils
 
 import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.executiongraph.ExecutionGraph
-import org.apache.flink.runtime.jobgraph.{JobStatus, JobID}
+import org.apache.flink.runtime.jobgraph.JobStatus
 
 object TestingJobManagerMessages {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
index 06062f4..a47e4e7 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
@@ -19,9 +19,9 @@
 package org.apache.flink.runtime.testingUtils
 
 import akka.actor.{Terminated, ActorRef}
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
 import org.apache.flink.runtime.instance.InstanceConnectionInfo
-import org.apache.flink.runtime.jobgraph.JobID
 import org.apache.flink.runtime.messages.Messages.Disconnect
 import org.apache.flink.runtime.messages.TaskManagerMessages.UnregisterTask
 import org.apache.flink.runtime.taskmanager.{NetworkEnvironmentConfiguration, TaskManagerConfiguration, TaskManager}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
index ebe4555..2051ef5 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
@@ -19,8 +19,8 @@
 package org.apache.flink.runtime.testingUtils
 
 import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
-import org.apache.flink.runtime.jobgraph.JobID
 import org.apache.flink.runtime.taskmanager.Task
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
index 0b394e3..a955682 100644
--- a/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
+++ b/flink-staging/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
@@ -52,7 +52,7 @@ public class AvroExternalJarProgramITCase {
 			PackagedProgram program = new PackagedProgram(new File(jarFile), new String[] { testData });
 						
 			Client c = new Client(new InetSocketAddress("localhost", testMiniCluster.getJobManagerRPCPort()),
-					new Configuration(), program.getUserCodeClassLoader());
+					new Configuration(), program.getUserCodeClassLoader(), -1);
 			c.run(program, 4, true);
 		}
 		catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-staging/flink-spargel/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-spargel/pom.xml b/flink-staging/flink-spargel/pom.xml
index a0f0f73..bd03c1a 100644
--- a/flink-staging/flink-spargel/pom.xml
+++ b/flink-staging/flink-spargel/pom.xml
@@ -56,5 +56,12 @@ under the License.
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-optimizer</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
 	</dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 38c26f0..d0b0164 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
@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -39,7 +40,6 @@ import org.apache.flink.runtime.operators.util.LocalStrategy;
 import org.apache.flink.spargel.java.examples.SpargelConnectedComponents.CCMessager;
 import org.apache.flink.spargel.java.examples.SpargelConnectedComponents.CCUpdater;
 import org.apache.flink.spargel.java.examples.SpargelConnectedComponents.IdAssigner;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
 
 
 public class SpargelCompilerTest extends CompilerTestBase {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 4faa329..3308ab7 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
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.client.program.Client;
 import org.apache.flink.client.program.JobWithJars;
 import org.apache.flink.client.program.ProgramInvocationException;
@@ -111,10 +112,16 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 
 		Configuration configuration = jobGraph.getJobConfiguration();
 		Client client = new Client(new InetSocketAddress(host, port), configuration,
-				JobWithJars.buildUserCodeClassLoader(jarFiles, JobWithJars.class.getClassLoader()));
+				JobWithJars.buildUserCodeClassLoader(jarFiles, JobWithJars.class.getClassLoader()), -1);
 
 		try {
-			return client.run(jobGraph, true);
+			JobSubmissionResult result = client.run(jobGraph, true);
+			if(result instanceof JobExecutionResult) {
+				return (JobExecutionResult) result;
+			} else {
+				LOG.warn("The Client didn't return a JobExecutionResult");
+				return new JobExecutionResult(result.getJobID(), -1, null);
+			}
 		} catch (ProgramInvocationException e) {
 			throw new RuntimeException("Cannot execute job due to ProgramInvocationException", e);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 b03ab0e..84ad710 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
@@ -21,15 +21,19 @@ import java.io.File;
 import java.util.List;
 
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.client.program.Client;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class StreamContextEnvironment extends StreamExecutionEnvironment {
 
-	protected static ClassLoader userClassLoader;
+	private static final Logger LOG = LoggerFactory.getLogger(StreamContextEnvironment.class);
+
 	protected List<File> jars;
 	protected Client client;
 
@@ -70,13 +74,13 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment {
 			jobGraph.addJar(new Path(file.getAbsolutePath()));
 		}
 
-		try {
-			return client.run(jobGraph, true);
-
-		} catch (Exception e) {
-			throw e;
+		JobSubmissionResult result = client.run(jobGraph, true);
+		if(result instanceof JobExecutionResult) {
+			return (JobExecutionResult) result;
+		} else {
+			LOG.warn("The Client didn't return a JobExecutionResult");
+			return new JobExecutionResult(result.getJobID(), -1, null);
 		}
-
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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
deleted file mode 100644
index 435713b..0000000
--- a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/CompilerTestBase.java
+++ /dev/null
@@ -1,201 +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.test.compiler.util;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
-import org.apache.flink.api.common.operators.GenericDataSourceBase;
-import org.apache.flink.api.common.operators.Operator;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.costs.DefaultCostEstimator;
-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.util.OperatingSystem;
-import org.junit.Before;
-
-/**
- *
- */
-public abstract class CompilerTestBase {
-
-	protected static final String IN_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/random";
-	
-	protected static final String OUT_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/null";
-	
-	protected static final int DEFAULT_PARALLELISM = 8;
-	
-	protected static final String DEFAULT_PARALLELISM_STRING = String.valueOf(DEFAULT_PARALLELISM);
-	
-	private static final String CACHE_KEY = "cachekey";
-	
-	// ------------------------------------------------------------------------
-	
-	protected DataStatistics dataStats;
-	
-	protected Optimizer withStatsCompiler;
-	
-	protected Optimizer noStatsCompiler;
-	
-	private int statCounter;
-	
-	// ------------------------------------------------------------------------	
-	
-	@Before
-	public void setup() {
-		this.dataStats = new DataStatistics();
-		this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator());
-		this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
-		
-		this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator());
-		this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	public OptimizedPlan compileWithStats(Plan p) {
-		return this.withStatsCompiler.compile(p);
-	}
-	
-	public OptimizedPlan compileNoStats(Plan p) {
-		return this.noStatsCompiler.compile(p);
-	}
-	
-	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) {
-		setSourceStatistics(source, new FileBaseStatistics(Long.MAX_VALUE, size, recordWidth));
-	}
-	
-	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, FileBaseStatistics stats) {
-		final String key = CACHE_KEY + this.statCounter++;
-		this.dataStats.cacheBaseStatistics(stats, key);
-		source.setStatisticsKey(key);
-	}
-
-	public static OperatorResolver getContractResolver(Plan plan) {
-		return new OperatorResolver(plan);
-	}
-	
-	public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) {
-		return new OptimizerPlanNodeResolver(plan);
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	public static final class OptimizerPlanNodeResolver {
-		
-		private final Map<String, ArrayList<PlanNode>> map;
-		
-		OptimizerPlanNodeResolver(OptimizedPlan p) {
-			HashMap<String, ArrayList<PlanNode>> map = new HashMap<String, ArrayList<PlanNode>>();
-			
-			for (PlanNode n : p.getAllNodes()) {
-				Operator<?> c = n.getOriginalOptimizerNode().getOperator();
-				String name = c.getName();
-				
-				ArrayList<PlanNode> list = map.get(name);
-				if (list == null) {
-					list = new ArrayList<PlanNode>(2);
-					map.put(name, list);
-				}
-				
-				// check whether this node is a child of a node with the same contract (aka combiner)
-				boolean shouldAdd = true;
-				for (Iterator<PlanNode> iter = list.iterator(); iter.hasNext();) {
-					PlanNode in = iter.next();
-					if (in.getOriginalOptimizerNode().getOperator() == c) {
-						// is this the child or is our node the child
-						if (in instanceof SingleInputPlanNode && n instanceof SingleInputPlanNode) {
-							SingleInputPlanNode thisNode = (SingleInputPlanNode) n;
-							SingleInputPlanNode otherNode = (SingleInputPlanNode) in;
-							
-							if (thisNode.getPredecessor() == otherNode) {
-								// other node is child, remove it
-								iter.remove();
-							} else if (otherNode.getPredecessor() == thisNode) {
-								shouldAdd = false;
-							}
-						} else {
-							throw new RuntimeException("Unrecodnized case in test.");
-						}
-					}
-				}
-				
-				if (shouldAdd) {
-					list.add(n);
-				}
-			}
-			
-			this.map = map;
-		}
-		
-		
-		@SuppressWarnings("unchecked")
-		public <T extends PlanNode> T getNode(String name) {
-			List<PlanNode> nodes = this.map.get(name);
-			if (nodes == null || nodes.isEmpty()) {
-				throw new RuntimeException("No node found with the given name.");
-			} else if (nodes.size() != 1) {
-				throw new RuntimeException("Multiple nodes found with the given name.");
-			} else {
-				return (T) nodes.get(0);
-			}
-		}
-		
-		@SuppressWarnings("unchecked")
-		public <T extends PlanNode> T getNode(String name, Class<? extends RichFunction> stubClass) {
-			List<PlanNode> nodes = this.map.get(name);
-			if (nodes == null || nodes.isEmpty()) {
-				throw new RuntimeException("No node found with the given name and stub class.");
-			} else {
-				PlanNode found = null;
-				for (PlanNode node : nodes) {
-					if (node.getClass() == stubClass) {
-						if (found == null) {
-							found = node;
-						} else {
-							throw new RuntimeException("Multiple nodes found with the given name and stub class.");
-						}
-					}
-				}
-				if (found == null) {
-					throw new RuntimeException("No node found with the given name and stub class.");
-				} else {
-					return (T) found;
-				}
-			}
-		}
-		
-		public List<PlanNode> getNodes(String name) {
-			List<PlanNode> nodes = this.map.get(name);
-			if (nodes == null || nodes.isEmpty()) {
-				throw new RuntimeException("No node found with the given name.");
-			} else {
-				return new ArrayList<PlanNode>(nodes);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java b/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java
deleted file mode 100644
index 08cded2..0000000
--- a/flink-test-utils/src/main/java/org/apache/flink/test/compiler/util/OperatorResolver.java
+++ /dev/null
@@ -1,127 +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.test.compiler.util;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.api.common.operators.Operator;
-import org.apache.flink.api.java.record.operators.BulkIteration;
-import org.apache.flink.api.java.record.operators.DeltaIteration;
-import org.apache.flink.util.Visitor;
-
-/**
- * Utility to get operator instances from plans via name.
- */
-@SuppressWarnings("deprecation")
-public class OperatorResolver implements Visitor<Operator<?>> {
-	
-	private final Map<String, List<Operator<?>>> map;
-	private Set<Operator<?>> seen;
-	
-	public OperatorResolver(Plan p) {
-		this.map = new HashMap<String, List<Operator<?>>>();
-		this.seen = new HashSet<Operator<?>>();
-		
-		p.accept(this);
-		this.seen = null;
-	}
-	
-	
-	@SuppressWarnings("unchecked")
-	public <T extends Operator<?>> T getNode(String name) {
-		List<Operator<?>> nodes = this.map.get(name);
-		if (nodes == null || nodes.isEmpty()) {
-			throw new RuntimeException("No nodes found with the given name.");
-		} else if (nodes.size() != 1) {
-			throw new RuntimeException("Multiple nodes found with the given name.");
-		} else {
-			return (T) nodes.get(0);
-		}
-	}
-	
-	@SuppressWarnings("unchecked")
-	public <T extends Operator<?>> T getNode(String name, Class<? extends RichFunction> stubClass) {
-		List<Operator<?>> nodes = this.map.get(name);
-		if (nodes == null || nodes.isEmpty()) {
-			throw new RuntimeException("No node found with the given name and stub class.");
-		} else {
-			Operator<?> found = null;
-			for (Operator<?> node : nodes) {
-				if (node.getClass() == stubClass) {
-					if (found == null) {
-						found = node;
-					} else {
-						throw new RuntimeException("Multiple nodes found with the given name and stub class.");
-					}
-				}
-			}
-			if (found == null) {
-				throw new RuntimeException("No node found with the given name and stub class.");
-			} else {
-				return (T) found;
-			}
-		}
-	}
-	
-	public List<Operator<?>> getNodes(String name) {
-		List<Operator<?>> nodes = this.map.get(name);
-		if (nodes == null || nodes.isEmpty()) {
-			throw new RuntimeException("No node found with the given name.");
-		} else {
-			return new ArrayList<Operator<?>>(nodes);
-		}
-	}
-
-	@Override
-	public boolean preVisit(Operator<?> visitable) {
-		if (this.seen.add(visitable)) {
-			// add to  the map
-			final String name = visitable.getName();
-			List<Operator<?>> list = this.map.get(name);
-			if (list == null) {
-				list = new ArrayList<Operator<?>>(2);
-				this.map.put(name, list);
-			}
-			list.add(visitable);
-			
-			// recurse into bulk iterations
-			if (visitable instanceof BulkIteration) {
-				((BulkIteration) visitable).getNextPartialSolution().accept(this);
-			} else if (visitable instanceof DeltaIteration) {
-				((DeltaIteration) visitable).getSolutionSetDelta().accept(this);
-				((DeltaIteration) visitable).getNextWorkset().accept(this);
-			}
-			
-			return true;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public void postVisit(Operator<?> visitable) {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 eafe9ad..5d4cf4d 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
@@ -72,7 +72,7 @@ public abstract class RecordAPITestBase extends AbstractTestBase {
 			Assert.fail("Error: Cannot obtain Pact plan. Did the thest forget to override either 'getPactPlan()' or 'getJobGraph()' ?");
 		}
 		
-		Optimizer pc = new Optimizer(new DataStatistics());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.config);
 		OptimizedPlan op = pc.compile(p);
 		
 		if (printPlan) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 02c1434..b3423cb 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
@@ -80,7 +80,7 @@ public class TestEnvironment extends ExecutionEnvironment {
 	private OptimizedPlan compileProgram(String jobName) {
 		Plan p = createProgramPlan(jobName);
 
-		Optimizer pc = new Optimizer(new DataStatistics());
+		Optimizer pc = new Optimizer(new DataStatistics(), this.executor.getConfiguration());
 		return pc.compile(p);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 9675150..977c205 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -126,6 +126,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-optimizer</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-runtime</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
index 400ed3a..345bffd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
@@ -140,7 +140,7 @@ public abstract class CancellingTestBase {
 	}
 
 	private JobGraph getJobGraph(final Plan plan) throws Exception {
-		final Optimizer pc = new Optimizer(new DataStatistics());
+		final Optimizer pc = new Optimizer(new DataStatistics(), this.executor.getConfiguration());
 		final OptimizedPlan op = pc.compile(plan);
 		final JobGraphGenerator jgg = new JobGraphGenerator();
 		return jgg.compileJobGraph(op);

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java b/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java
index 4fd4617..1724920 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/compiler/examples/KMeansSingleStepTest.java
@@ -31,8 +31,8 @@ import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.test.compiler.util.CompilerTestBase;
-import org.apache.flink.test.compiler.util.OperatorResolver;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.OperatorResolver;
 import org.apache.flink.test.recordJobs.kmeans.KMeansSingleStep;
 import org.junit.Assert;
 import org.junit.Test;


[3/4] flink git commit: [FLINK-1771] Add support for submitting single jobs to a detached YARN session

Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 3cf081f..d6b9444 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
@@ -35,6 +35,7 @@ import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.plan.SourcePlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java
index f6885c5..3af64fc 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java
@@ -26,6 +26,7 @@ import org.apache.flink.api.java.record.operators.FileDataSource;
 import org.apache.flink.api.java.record.operators.ReduceOperator;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.optimizer.util.DummyInputFormat;
 import org.apache.flink.optimizer.util.DummyOutputFormat;
 import org.apache.flink.optimizer.util.IdentityReduce;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 230cc6b..25643a4 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
@@ -39,6 +39,7 @@ import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
index 1fe16bb..3a24ce1 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
@@ -38,6 +38,7 @@ 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.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.util.Visitor;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java
index 40b54e0..65e5025 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.optimizer;
 
 import static org.junit.Assert.*;
 
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.functions.Partitioner;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java
index 92b4fc5..1001626 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java
index 5d15ed8..2e52565 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.java.record.operators.FileDataSource;
 import org.apache.flink.api.java.record.operators.ReduceOperator;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.Visitor;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java
index 1e4124c..e81e0ec 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 import static org.junit.Assert.fail;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java
index 80c0bda..321ca5a 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java
@@ -29,6 +29,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java
index 6e7c0a3..27f367f 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java
@@ -42,6 +42,7 @@ 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.optimizer.util.CompilerTestBase;
 import org.apache.flink.types.LongValue;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java
index 0273659..b4e95fb 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java
@@ -26,13 +26,13 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 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.testfunctions.DummyCoGroupFunction;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java
index 08f7388..d52181d 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java
@@ -29,13 +29,13 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction;
 import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
 import org.apache.flink.optimizer.testfunctions.IdentityMapper;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java
index 9fd676f..5758c86 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java
@@ -28,12 +28,12 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 f865a9f..00fd587 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
@@ -27,11 +27,11 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.IdentityPartitionerMapper;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java
index 360487b..0408ca9 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java
@@ -30,12 +30,12 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.DummyReducer;
 import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java
index 8cd4809..74e5c8c 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java
@@ -26,12 +26,12 @@ import org.apache.flink.api.common.functions.Partitioner;
 import org.apache.flink.api.common.operators.Order;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.DummyReducer;
 import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java
index 779b8e5..72fb81b 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java
@@ -29,12 +29,12 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.DummyReducer;
 import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java
index eae40cf..8eedee1 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java
@@ -30,13 +30,13 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction;
 import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
 import org.apache.flink.optimizer.testfunctions.IdentityMapper;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java
index cb4bd78..5f69336 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java
@@ -25,7 +25,6 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
@@ -35,6 +34,7 @@ import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction;
 import org.apache.flink.optimizer.testfunctions.IdentityFlatMapper;
 import org.apache.flink.optimizer.testfunctions.SelectOneReducer;
 import org.apache.flink.optimizer.testfunctions.Top1GroupReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.io.network.DataExchangeMode;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java
index 5175d8c..13ec51a 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java
@@ -24,12 +24,12 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor;
 import org.apache.flink.optimizer.testfunctions.Top1GroupReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.io.network.DataExchangeMode;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java
index 6b2691a..99f8c81 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java
@@ -25,11 +25,11 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.io.network.DataExchangeMode;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java
index b359e6b..ab83dba 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.optimizer.java;
 
 import static org.junit.Assert.fail;
 
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 import org.apache.flink.api.common.Plan;
@@ -28,7 +29,6 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.optimizer.CompilerException;
-import org.apache.flink.optimizer.CompilerTestBase;
 
 
 @SuppressWarnings({"serial", "unchecked"})

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 2f9b32f..96758b1 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
@@ -20,12 +20,12 @@ package org.apache.flink.optimizer.java;
 
 import static org.junit.Assert.*;
 
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 c0e2fa7..1bd4b8a 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
@@ -24,12 +24,12 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.operators.GroupReduceOperator;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 57d2d54..796d4ab 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
@@ -27,7 +27,6 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.BulkIterationPlanNode;
 import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
@@ -36,6 +35,7 @@ import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.optimizer.testfunctions.IdentityMapper;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java
index 0a62132..14d863d 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java
@@ -27,10 +27,10 @@ import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.util.Visitor;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.java
index cd63b72..3f18e62 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.java
@@ -27,8 +27,8 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.testfunctions.IdentityMapper;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java
index 8bb9a76..95ee4de 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java
@@ -27,11 +27,11 @@ import org.apache.flink.api.common.functions.Partitioner;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 e1b18f9..4197abb 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
@@ -23,10 +23,10 @@ import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 f1c2233..46eb48a 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
@@ -33,11 +33,11 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.common.functions.RichJoinFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.util.Collector;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java
index e7807c9..fb7a80f 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java
@@ -25,10 +25,10 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.optimizer.CompilerException;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings({"serial", "unchecked"})

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java
index 9171cc7..8a4786f 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java
@@ -25,9 +25,9 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.optimizer.CompilerException;
-import org.apache.flink.optimizer.CompilerTestBase;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings({"serial", "unchecked"})

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java
new file mode 100644
index 0000000..35c50d3
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java
@@ -0,0 +1,240 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
+import org.apache.flink.api.common.operators.GenericDataSourceBase;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.base.BulkIterationBase;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.optimizer.DataStatistics;
+import org.apache.flink.optimizer.Optimizer;
+import org.apache.flink.optimizer.costs.DefaultCostEstimator;
+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.util.OperatingSystem;
+import org.apache.flink.util.Visitor;
+import org.junit.Before;
+
+/**
+ * Base class for Optimizer tests. Offers utility methods to trigger optimization
+ * of a program and to fetch the nodes in an optimizer plan that correspond
+ * the the node in the program plan.
+ */
+public abstract class CompilerTestBase implements java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	protected static final String IN_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/random";
+
+	protected static final String OUT_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/null";
+
+	protected static final int DEFAULT_PARALLELISM = 8;
+
+	protected static final String DEFAULT_PARALLELISM_STRING = String.valueOf(DEFAULT_PARALLELISM);
+
+	private static final String CACHE_KEY = "cachekey";
+
+	// ------------------------------------------------------------------------
+
+	protected transient DataStatistics dataStats;
+
+	protected transient Optimizer withStatsCompiler;
+
+	protected transient Optimizer noStatsCompiler;
+
+	private transient int statCounter;
+
+	// ------------------------------------------------------------------------
+
+	@Before
+	public void setup() {
+		Configuration flinkConf = new Configuration();
+		this.dataStats = new DataStatistics();
+		this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator(), flinkConf);
+		this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
+
+		this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator(), flinkConf);
+		this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
+	}
+
+	// ------------------------------------------------------------------------
+
+	public OptimizedPlan compileWithStats(Plan p) {
+		return this.withStatsCompiler.compile(p);
+	}
+
+	public OptimizedPlan compileNoStats(Plan p) {
+		return this.noStatsCompiler.compile(p);
+	}
+
+	public static OperatorResolver getContractResolver(Plan plan) {
+		return new OperatorResolver(plan);
+	}
+
+	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) {
+		setSourceStatistics(source, new FileBaseStatistics(Long.MAX_VALUE, size, recordWidth));
+	}
+
+	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, FileBaseStatistics stats) {
+		final String key = CACHE_KEY + this.statCounter++;
+		this.dataStats.cacheBaseStatistics(stats, key);
+		source.setStatisticsKey(key);
+	}
+
+
+	public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) {
+		return new OptimizerPlanNodeResolver(plan);
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static final class OptimizerPlanNodeResolver {
+
+		private final Map<String, ArrayList<PlanNode>> map;
+
+		public OptimizerPlanNodeResolver(OptimizedPlan p) {
+			HashMap<String, ArrayList<PlanNode>> map = new HashMap<String, ArrayList<PlanNode>>();
+
+			for (PlanNode n : p.getAllNodes()) {
+				Operator<?> c = n.getOriginalOptimizerNode().getOperator();
+				String name = c.getName();
+
+				ArrayList<PlanNode> list = map.get(name);
+				if (list == null) {
+					list = new ArrayList<PlanNode>(2);
+					map.put(name, list);
+				}
+
+				// check whether this node is a child of a node with the same contract (aka combiner)
+				boolean shouldAdd = true;
+				for (Iterator<PlanNode> iter = list.iterator(); iter.hasNext();) {
+					PlanNode in = iter.next();
+					if (in.getOriginalOptimizerNode().getOperator() == c) {
+						// is this the child or is our node the child
+						if (in instanceof SingleInputPlanNode && n instanceof SingleInputPlanNode) {
+							SingleInputPlanNode thisNode = (SingleInputPlanNode) n;
+							SingleInputPlanNode otherNode = (SingleInputPlanNode) in;
+
+							if (thisNode.getPredecessor() == otherNode) {
+								// other node is child, remove it
+								iter.remove();
+							} else if (otherNode.getPredecessor() == thisNode) {
+								shouldAdd = false;
+							}
+						} else {
+							throw new RuntimeException("Unrecodnized case in test.");
+						}
+					}
+				}
+
+				if (shouldAdd) {
+					list.add(n);
+				}
+			}
+
+			this.map = map;
+		}
+
+
+		@SuppressWarnings("unchecked")
+		public <T extends PlanNode> T getNode(String name) {
+			List<PlanNode> nodes = this.map.get(name);
+			if (nodes == null || nodes.isEmpty()) {
+				throw new RuntimeException("No node found with the given name.");
+			} else if (nodes.size() != 1) {
+				throw new RuntimeException("Multiple nodes found with the given name.");
+			} else {
+				return (T) nodes.get(0);
+			}
+		}
+
+		@SuppressWarnings("unchecked")
+		public <T extends PlanNode> T getNode(String name, Class<? extends Function> stubClass) {
+			List<PlanNode> nodes = this.map.get(name);
+			if (nodes == null || nodes.isEmpty()) {
+				throw new RuntimeException("No node found with the given name and stub class.");
+			} else {
+				PlanNode found = null;
+				for (PlanNode node : nodes) {
+					if (node.getClass() == stubClass) {
+						if (found == null) {
+							found = node;
+						} else {
+							throw new RuntimeException("Multiple nodes found with the given name and stub class.");
+						}
+					}
+				}
+				if (found == null) {
+					throw new RuntimeException("No node found with the given name and stub class.");
+				} else {
+					return (T) found;
+				}
+			}
+		}
+
+		public List<PlanNode> getNodes(String name) {
+			List<PlanNode> nodes = this.map.get(name);
+			if (nodes == null || nodes.isEmpty()) {
+				throw new RuntimeException("No node found with the given name.");
+			} else {
+				return new ArrayList<PlanNode>(nodes);
+			}
+		}
+	}
+
+	/**
+	 * Collects all DataSources of a plan to add statistics
+	 *
+	 */
+	public static class SourceCollectorVisitor implements Visitor<Operator<?>> {
+
+		protected final List<GenericDataSourceBase<?, ?>> sources = new ArrayList<GenericDataSourceBase<?, ?>>(4);
+
+		@Override
+		public boolean preVisit(Operator<?> visitable) {
+
+			if(visitable instanceof GenericDataSourceBase) {
+				sources.add((GenericDataSourceBase<?, ?>) visitable);
+			}
+			else if(visitable instanceof BulkIterationBase) {
+				((BulkIterationBase<?>) visitable).getNextPartialSolution().accept(this);
+			}
+
+			return true;
+		}
+
+		@Override
+		public void postVisit(Operator<?> visitable) {}
+
+		public List<GenericDataSourceBase<?, ?>> getSources() {
+			return this.sources;
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/OperatorResolver.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/OperatorResolver.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/OperatorResolver.java
new file mode 100644
index 0000000..920b713
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/OperatorResolver.java
@@ -0,0 +1,127 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.java.record.operators.BulkIteration;
+import org.apache.flink.api.java.record.operators.DeltaIteration;
+import org.apache.flink.util.Visitor;
+
+/**
+ * Utility to get operator instances from plans via name.
+ */
+@SuppressWarnings("deprecation")
+public class OperatorResolver implements Visitor<Operator<?>> {
+	
+	private final Map<String, List<Operator<?>>> map;
+	private Set<Operator<?>> seen;
+	
+	public OperatorResolver(Plan p) {
+		this.map = new HashMap<String, List<Operator<?>>>();
+		this.seen = new HashSet<Operator<?>>();
+		
+		p.accept(this);
+		this.seen = null;
+	}
+	
+	
+	@SuppressWarnings("unchecked")
+	public <T extends Operator<?>> T getNode(String name) {
+		List<Operator<?>> nodes = this.map.get(name);
+		if (nodes == null || nodes.isEmpty()) {
+			throw new RuntimeException("No nodes found with the given name.");
+		} else if (nodes.size() != 1) {
+			throw new RuntimeException("Multiple nodes found with the given name.");
+		} else {
+			return (T) nodes.get(0);
+		}
+	}
+	
+	@SuppressWarnings("unchecked")
+	public <T extends Operator<?>> T getNode(String name, Class<? extends RichFunction> stubClass) {
+		List<Operator<?>> nodes = this.map.get(name);
+		if (nodes == null || nodes.isEmpty()) {
+			throw new RuntimeException("No node found with the given name and stub class.");
+		} else {
+			Operator<?> found = null;
+			for (Operator<?> node : nodes) {
+				if (node.getClass() == stubClass) {
+					if (found == null) {
+						found = node;
+					} else {
+						throw new RuntimeException("Multiple nodes found with the given name and stub class.");
+					}
+				}
+			}
+			if (found == null) {
+				throw new RuntimeException("No node found with the given name and stub class.");
+			} else {
+				return (T) found;
+			}
+		}
+	}
+	
+	public List<Operator<?>> getNodes(String name) {
+		List<Operator<?>> nodes = this.map.get(name);
+		if (nodes == null || nodes.isEmpty()) {
+			throw new RuntimeException("No node found with the given name.");
+		} else {
+			return new ArrayList<Operator<?>>(nodes);
+		}
+	}
+
+	@Override
+	public boolean preVisit(Operator<?> visitable) {
+		if (this.seen.add(visitable)) {
+			// add to  the map
+			final String name = visitable.getName();
+			List<Operator<?>> list = this.map.get(name);
+			if (list == null) {
+				list = new ArrayList<Operator<?>>(2);
+				this.map.put(name, list);
+			}
+			list.add(visitable);
+			
+			// recurse into bulk iterations
+			if (visitable instanceof BulkIteration) {
+				((BulkIteration) visitable).getNextPartialSolution().accept(this);
+			} else if (visitable instanceof DeltaIteration) {
+				((DeltaIteration) visitable).getSolutionSetDelta().accept(this);
+				((DeltaIteration) visitable).getNextWorkset().accept(this);
+			}
+			
+			return true;
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public void postVisit(Operator<?> visitable) {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-quickstart/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/pom.xml b/flink-quickstart/pom.xml
index 78b99ad..4f3b4b2 100644
--- a/flink-quickstart/pom.xml
+++ b/flink-quickstart/pom.xml
@@ -59,7 +59,7 @@ under the License.
 				<artifactId>maven-archetype-plugin</artifactId>
 				<version>2.2</version><!--$NO-MVN-MAN-VER$-->
 				<configuration>
-					<skip>${skipTests}</skip>
+					<skip>true</skip>
 				</configuration>
 			</plugin>
 			<!-- deactivate the shade plugin for the quickstart archetypes -->

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorEvent.java
index fd91d65..f345d6a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorEvent.java
@@ -29,7 +29,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.util.InstantiationUtil;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
index cb799c4..1c76e08 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java
@@ -32,7 +32,7 @@ import org.apache.flink.util.InstantiationUtil;
 import org.slf4j.LoggerFactory;
 import org.slf4j.Logger;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE;
 import static org.apache.flink.runtime.blob.BlobServerProtocol.JOB_ID_SCOPE;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
index 78e9707..626d21f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java
@@ -33,7 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
index b2f11db..50b1f24 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java
@@ -29,7 +29,7 @@ import java.net.Socket;
 import java.net.SocketException;
 import java.security.MessageDigest;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.util.InstantiationUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
index 5db5ef6..69687da 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.blob;
 
 import com.google.common.io.BaseEncoding;
 import org.apache.commons.io.FileUtils;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.slf4j.Logger;
 
 import java.io.EOFException;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobCancellationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobCancellationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobCancellationException.java
index 9f72db0..842870d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobCancellationException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobCancellationException.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.client;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * An exception which is thrown by the JobClient if a job is aborted as a result of a user

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java
index 99c3f89..56ccef5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.client;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * This exception is the base exception for all exceptions that denote any failure during

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobStatusMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobStatusMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobStatusMessage.java
index d2c81a5..c33ed8f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobStatusMessage.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobStatusMessage.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.client;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionException.java
index 3d672a5..3cb0b9f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionException.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.client;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * This exception denotes an error while submitting a job to the JobManager

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobTimeoutException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobTimeoutException.java
index 10ef601..18c8c31 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobTimeoutException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobTimeoutException.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.client;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * An exception which is thrown by the JobClient if the job manager is no longer reachable.

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
index f7518bd..5d96903 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.deployment;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.StateHandle;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
index 556bb11..503a0b9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memorymanager.MemoryManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
index c6270b2..3fb7493 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
@@ -33,7 +33,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
index d394e2d..848f619 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java
@@ -35,7 +35,7 @@ import java.util.TimerTask;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.blob.BlobService;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.util.ExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java
index 532107f..66bda45 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.execution.librarycache;
 
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
index 63d85b4..52a8048 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.execution.librarycache;
 
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 import java.io.File;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 d0615b3..5ce89b3 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
@@ -28,7 +28,7 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
index ad72d13..acbc17a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -29,7 +29,7 @@ import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobEdge;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index 8dd341f..1b91089 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -34,7 +34,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.jobgraph.JobEdge;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
index b838aa4..3449100 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
@@ -38,7 +38,7 @@ import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.fs.local.LocalFileSystem;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.util.IOUtils;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
index e27b7ea..324629f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
@@ -27,7 +27,7 @@ import java.util.Set;
 
 import akka.actor.ActorRef;
 import org.apache.flink.util.AbstractID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroupAssignment;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java
index eab4344..7bd70a7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.instance;
 
 import org.apache.flink.util.AbstractID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroupAssignment;
 
 import java.util.HashSet;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java
index c89b7f5..56be9d0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.instance;
 
 import org.apache.flink.util.AbstractID;
 import org.apache.flink.runtime.executiongraph.Execution;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobmanager.scheduler.Locality;
 
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
index 08d6441..bf8464c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.instance;
 
 import org.apache.flink.util.AbstractID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
index d490784..55b89b4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
@@ -22,6 +22,7 @@ import akka.actor.ActorRef;
 import akka.dispatch.OnFailure;
 import akka.pattern.Patterns;
 import akka.util.Timeout;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
@@ -35,7 +36,6 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
-import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
index 92e27d3..88020a4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
@@ -28,7 +28,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel;
 import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionConsumableNotifier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionConsumableNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionConsumableNotifier.java
index 0ea3a1c..6e84b4c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionConsumableNotifier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionConsumableNotifier.java
@@ -18,7 +18,8 @@
 
 package org.apache.flink.runtime.io.network.partition;
 
-import org.apache.flink.runtime.jobgraph.JobID;
+
+import org.apache.flink.api.common.JobID;
 
 public interface ResultPartitionConsumableNotifier {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index f8f22b0..d43533b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.fs.FileSystem;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java
deleted file mode 100644
index 7c8d365..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java
+++ /dev/null
@@ -1,61 +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.runtime.jobgraph;
-
-import javax.xml.bind.DatatypeConverter;
-
-import org.apache.flink.util.AbstractID;
-
-import java.nio.ByteBuffer;
-
-public final class JobID extends AbstractID {
-
-	private static final long serialVersionUID = 1L;
-	
-	public JobID() {
-		super();
-	}
-
-	public JobID(long lowerPart, long upperPart) {
-		super(lowerPart, upperPart);
-	}
-
-	public JobID(byte[] bytes) {
-		super(bytes);
-	}
-
-	public static JobID generate() {
-		return new JobID();
-	}
-
-	public static JobID fromByteArray(byte[] bytes) {
-		return new JobID(bytes);
-	}
-
-	public static JobID fromByteBuffer(ByteBuffer buf) {
-		long lower = buf.getLong();
-		long upper = buf.getLong();
-		return new JobID(lower, upper);
-	}
-
-	public static JobID fromHexString(String hexString) {
-		return new JobID(DatatypeConverter.parseHexBinary(hexString));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java
index 3130354..5bf8ebe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/accumulators/AccumulatorManager.java
@@ -24,7 +24,7 @@ import java.util.LinkedList;
 import java.util.Map;
 
 import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * This class manages the accumulators for different jobs. Either the jobs are

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java
index ffb0dd4..008be65 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobManagerInfoServlet.java
@@ -56,7 +56,7 @@ import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.util.EnvironmentInformation;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/JobManagerProfiler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/JobManagerProfiler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/JobManagerProfiler.java
index 8f16e4f..4c4a20e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/JobManagerProfiler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/JobManagerProfiler.java
@@ -20,7 +20,7 @@
 package org.apache.flink.runtime.profiling;
 
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 
 /**
  * This interface must be implemented by profiling components

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
index 85dedb7..d5ce137 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
@@ -25,7 +25,7 @@ import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.profiling.impl.types.InternalExecutionVertexThreadProfilingData;
 


[4/4] flink git commit: [FLINK-1771] Add support for submitting single jobs to a detached YARN session

Posted by rm...@apache.org.
[FLINK-1771] Add support for submitting single jobs to a detached YARN session

With this change, users can submit a Flink job to a YARN cluster without having a local client monitoring the Application Master or job. You can basically fire and forget a Flink job to YARN.
For supporting this, the ApplicationMaster can now monitor the status of a job and shutdown itself once it is in a terminal state.

The change also verifies that various ways of setting the parallelism on YARN are passed through the system correctly (per job, session).

There was a bug in YARN container creation which made the configuration values for the heap offset useless. This change fixes this error.

All mentioned features and bugs are covered by the flink-yarn-tests.

This closes #542


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

Branch: refs/heads/master
Commit: 6b0d40764da9dce2e2d21882e9a03a21c6783ff0
Parents: 121a5a0
Author: Robert Metzger <rm...@apache.org>
Authored: Fri Mar 13 15:53:51 2015 +0100
Committer: Robert Metzger <rm...@apache.org>
Committed: Tue Mar 31 08:56:39 2015 +0200

----------------------------------------------------------------------
 .../org/apache/flink/client/CliFrontend.java    | 116 ++++++---
 .../flink/client/FlinkYarnSessionCli.java       |  12 +-
 .../org/apache/flink/client/LocalExecutor.java  |  19 +-
 .../org/apache/flink/client/RemoteExecutor.java |  27 ++-
 .../org/apache/flink/client/program/Client.java |  85 ++++---
 .../client/program/ContextEnvironment.java      |  39 ++-
 .../flink/client/CliFrontendInfoTest.java       |   4 +-
 .../flink/client/CliFrontendListCancelTest.java |   2 +-
 .../apache/flink/client/CliFrontendRunTest.java |   2 +-
 .../client/program/ClientConnectionTest.java    |   2 +-
 .../apache/flink/client/program/ClientTest.java |   9 +-
 .../ExecutionPlanAfterExecutionTest.java        |   3 +-
 .../program/ExecutionPlanCreationTest.java      |   2 +-
 .../flink/api/common/JobExecutionResult.java    |   7 +-
 .../java/org/apache/flink/api/common/JobID.java |  60 +++++
 .../flink/api/common/JobSubmissionResult.java   |  39 +++
 .../common/operators/CollectionExecutor.java    |   2 +-
 .../main/flink-bin/conf/log4j-cli.properties    |   1 +
 .../examples/java/wordcount/WordCount.java      |   1 -
 flink-optimizer/pom.xml                         |  19 +-
 .../org/apache/flink/optimizer/Optimizer.java   |  20 +-
 .../optimizer/AdditionalOperatorsTest.java      |   1 +
 .../optimizer/BranchingPlansCompilerTest.java   |   1 +
 .../BroadcastVariablePipelinebreakerTest.java   |   1 +
 .../CachedMatchStrategyCompilerTest.java        |   1 +
 .../optimizer/CoGroupSolutionSetFirstTest.java  |   1 +
 .../flink/optimizer/CompilerTestBase.java       | 229 ------------------
 .../flink/optimizer/DisjointDataFlowsTest.java  |   1 +
 .../optimizer/DistinctCompilationTest.java      |   1 +
 .../apache/flink/optimizer/GroupOrderTest.java  |   1 +
 .../optimizer/HardPlansCompilationTest.java     |   1 +
 .../flink/optimizer/IterationsCompilerTest.java |   1 +
 .../flink/optimizer/NestedIterationsTest.java   |   1 +
 .../flink/optimizer/ParallelismChangeTest.java  |   1 +
 .../flink/optimizer/PartitionPushdownTest.java  |   1 +
 .../optimizer/PartitioningReusageTest.java      |   1 +
 .../flink/optimizer/PipelineBreakerTest.java    |   1 +
 .../flink/optimizer/PropertyDataSourceTest.java |   1 +
 .../apache/flink/optimizer/ReduceAllTest.java   |   1 +
 .../optimizer/ReplicatingDataSourceTest.java    |   1 +
 .../SemanticPropertiesAPIToPlanTest.java        |   1 +
 .../flink/optimizer/SortPartialReuseTest.java   |   1 +
 .../UnionBetweenDynamicAndStaticPathTest.java   |   1 +
 .../optimizer/UnionPropertyPropagationTest.java |   1 +
 .../flink/optimizer/UnionReplacementTest.java   |   1 +
 .../WorksetIterationCornerCasesTest.java        |   1 +
 .../WorksetIterationsRecordApiCompilerTest.java |   1 +
 ...naryCustomPartitioningCompatibilityTest.java |   2 +-
 .../CoGroupCustomPartitioningTest.java          |   2 +-
 ...ustomPartitioningGlobalOptimizationTest.java |   2 +-
 .../custompartition/CustomPartitioningTest.java |   2 +-
 .../GroupingKeySelectorTranslationTest.java     |   2 +-
 .../GroupingPojoTranslationTest.java            |   2 +-
 .../GroupingTupleTranslationTest.java           |   2 +-
 .../JoinCustomPartitioningTest.java             |   2 +-
 .../DataExchangeModeClosedBranchingTest.java    |   2 +-
 .../DataExchangeModeForwardTest.java            |   2 +-
 .../DataExchangeModeOpenBranchingTest.java      |   2 +-
 .../java/DeltaIterationDependenciesTest.java    |   2 +-
 .../java/DistinctAndGroupingOptimizerTest.java  |   2 +-
 .../java/GroupReduceCompilationTest.java        |   2 +-
 .../optimizer/java/IterationCompilerTest.java   |   2 +-
 .../optimizer/java/JoinTranslationTest.java     |   2 +-
 .../flink/optimizer/java/OpenIterationTest.java |   2 +-
 .../optimizer/java/PartitionOperatorTest.java   |   2 +-
 .../optimizer/java/ReduceCompilationTest.java   |   2 +-
 .../WorksetIterationsJavaApiCompilerTest.java   |   2 +-
 .../CoGroupOnConflictingPartitioningsTest.java  |   2 +-
 .../JoinOnConflictingPartitioningsTest.java     |   2 +-
 .../flink/optimizer/util/CompilerTestBase.java  | 240 +++++++++++++++++++
 .../flink/optimizer/util/OperatorResolver.java  | 127 ++++++++++
 flink-quickstart/pom.xml                        |   2 +-
 .../runtime/accumulators/AccumulatorEvent.java  |   2 +-
 .../apache/flink/runtime/blob/BlobClient.java   |   2 +-
 .../apache/flink/runtime/blob/BlobServer.java   |   2 +-
 .../runtime/blob/BlobServerConnection.java      |   2 +-
 .../apache/flink/runtime/blob/BlobUtils.java    |   2 +-
 .../client/JobCancellationException.java        |   2 +-
 .../runtime/client/JobExecutionException.java   |   2 +-
 .../flink/runtime/client/JobStatusMessage.java  |   2 +-
 .../runtime/client/JobSubmissionException.java  |   2 +-
 .../runtime/client/JobTimeoutException.java     |   2 +-
 .../deployment/TaskDeploymentDescriptor.java    |   2 +-
 .../flink/runtime/execution/Environment.java    |   2 +-
 .../runtime/execution/RuntimeEnvironment.java   |   2 +-
 .../librarycache/BlobLibraryCacheManager.java   |   2 +-
 .../FallbackLibraryCacheManager.java            |   2 +-
 .../librarycache/LibraryCacheManager.java       |   2 +-
 .../runtime/executiongraph/ExecutionGraph.java  |   2 +-
 .../executiongraph/ExecutionJobVertex.java      |   2 +-
 .../runtime/executiongraph/ExecutionVertex.java |   2 +-
 .../flink/runtime/filecache/FileCache.java      |   2 +-
 .../apache/flink/runtime/instance/Instance.java |   2 +-
 .../flink/runtime/instance/SharedSlot.java      |   2 +-
 .../flink/runtime/instance/SimpleSlot.java      |   2 +-
 .../org/apache/flink/runtime/instance/Slot.java |   2 +-
 .../runtime/io/network/NetworkEnvironment.java  |   2 +-
 .../io/network/partition/ResultPartition.java   |   2 +-
 .../ResultPartitionConsumableNotifier.java      |   3 +-
 .../apache/flink/runtime/jobgraph/JobGraph.java |   1 +
 .../apache/flink/runtime/jobgraph/JobID.java    |  61 -----
 .../accumulators/AccumulatorManager.java        |   2 +-
 .../jobmanager/web/JobManagerInfoServlet.java   |   2 +-
 .../runtime/profiling/JobManagerProfiler.java   |   2 +-
 .../profiling/impl/EnvironmentThreadSet.java    |   2 +-
 .../InternalExecutionVertexProfilingData.java   |   2 +-
 ...ernalExecutionVertexThreadProfilingData.java |   2 +-
 .../profiling/types/InstanceProfilingEvent.java |   2 +-
 .../types/InstanceSummaryProfilingEvent.java    |   2 +-
 .../runtime/profiling/types/ProfilingEvent.java |   2 +-
 .../types/SingleInstanceProfilingEvent.java     |   2 +-
 .../profiling/types/ThreadProfilingEvent.java   |   2 +-
 .../profiling/types/VertexProfilingEvent.java   |   2 +-
 .../apache/flink/runtime/taskmanager/Task.java  |   2 +-
 .../runtime/taskmanager/TaskExecutionState.java |   2 +-
 .../taskmanager/TaskInputSplitProvider.java     |   2 +-
 .../runtime/yarn/AbstractFlinkYarnClient.java   |  84 ++++++-
 .../runtime/yarn/AbstractFlinkYarnCluster.java  |  29 +++
 .../apache/flink/runtime/client/JobClient.scala |   4 +-
 .../flink/runtime/jobmanager/JobManager.scala   |   6 +-
 .../runtime/jobmanager/MemoryArchivist.scala    |   2 +-
 .../StreamCheckpointCoordinator.scala           |   3 +-
 .../runtime/messages/ArchiveMessages.scala      |   2 +-
 .../messages/ExecutionGraphMessages.scala       |   3 +-
 .../runtime/messages/JobManagerMessages.scala   |   3 +-
 .../minicluster/LocalFlinkMiniCluster.scala     |   4 +
 .../flink/runtime/blob/BlobClientTest.java      |   2 +-
 .../runtime/blob/BlobServerDeleteTest.java      |   2 +-
 .../flink/runtime/blob/BlobServerPutTest.java   |   2 +-
 .../TaskDeploymentDescriptorTest.java           |   2 +-
 .../BlobLibraryCacheManagerTest.java            |   2 +-
 .../ExecutionGraphConstructionTest.java         |   2 +-
 .../ExecutionGraphDeploymentTest.java           |   2 +-
 .../executiongraph/ExecutionGraphTestUtils.java |   2 +-
 .../ExecutionStateProgressTest.java             |   2 +-
 .../ExecutionVertexCancelTest.java              |   2 +-
 .../ExecutionVertexDeploymentTest.java          |   2 +-
 .../ExecutionVertexSchedulingTest.java          |   2 +-
 .../executiongraph/PointwisePatternTest.java    |   2 +-
 .../executiongraph/VertexSlotSharingTest.java   |   2 +-
 .../FileCacheDeleteValidationTest.java          |   2 +-
 .../flink/runtime/instance/InstanceTest.java    |   2 +-
 .../flink/runtime/instance/SimpleSlotTest.java  |   2 +-
 .../consumer/LocalInputChannelTest.java         |   2 +-
 .../flink/runtime/jobgraph/JobIdTest.java       |   1 +
 .../scheduler/SchedulerTestUtils.java           |   2 +-
 .../jobmanager/scheduler/SharedSlotsTest.java   |   2 +-
 .../scheduler/SlotAllocationFutureTest.java     |   2 +-
 .../operators/testutils/MockEnvironment.java    |   2 +-
 .../profiling/types/ProfilingTypesTest.java     |   5 +-
 .../taskmanager/TaskExecutionStateTest.java     |   2 +-
 .../runtime/taskmanager/TaskManagerTest.java    |   2 +-
 .../flink/runtime/taskmanager/TaskTest.java     |   2 +-
 .../ExecutionGraphRestartTest.scala             |   3 +-
 .../TaskManagerLossFailsTasksTest.scala         |   3 +-
 .../testingUtils/TestingJobManager.scala        |   3 +-
 .../TestingJobManagerMessages.scala             |   3 +-
 .../testingUtils/TestingTaskManager.scala       |   2 +-
 .../TestingTaskManagerMessages.scala            |   2 +-
 .../api/avro/AvroExternalJarProgramITCase.java  |   2 +-
 flink-staging/flink-spargel/pom.xml             |   7 +
 .../flink/spargel/java/SpargelCompilerTest.java |   2 +-
 .../environment/RemoteStreamEnvironment.java    |  11 +-
 .../environment/StreamContextEnvironment.java   |  18 +-
 .../test/compiler/util/CompilerTestBase.java    | 201 ----------------
 .../test/compiler/util/OperatorResolver.java    | 127 ----------
 .../flink/test/util/RecordAPITestBase.java      |   2 +-
 .../apache/flink/test/util/TestEnvironment.java |   2 +-
 flink-tests/pom.xml                             |   8 +
 .../test/cancelling/CancellingTestBase.java     |   2 +-
 .../compiler/examples/KMeansSingleStepTest.java |   4 +-
 .../examples/RelationalQueryCompilerTest.java   |   4 +-
 .../examples/WordCountCompilerTest.java         |   2 +-
 .../ConnectedComponentsCoGroupTest.java         |   2 +-
 .../iterations/ConnectedComponentsTest.java     |   2 +-
 .../iterations/IterativeKMeansTest.java         |   4 +-
 ...ultipleJoinsWithSolutionSetCompilerTest.java |   2 +-
 .../iterations/PageRankCompilerTest.java        |   2 +-
 .../compiler/plandump/DumpCompiledPlanTest.java |   2 +-
 .../test/failingPrograms/TaskFailureITCase.java |   4 +-
 .../flink/test/operators/ReduceITCase.java      |   2 +-
 .../flink/test/operators/UnionSinkITCase.java   |   2 +-
 .../PartitionOperatorTranslationTest.scala      |   2 +-
 .../CoGroupCustomPartitioningTest.scala         |   2 +-
 .../CoGroupGroupSortTranslationTest.scala       |   2 +-
 ...tomPartitioningGroupingKeySelectorTest.scala |   2 +-
 .../CustomPartitioningGroupingPojoTest.scala    |   2 +-
 .../CustomPartitioningGroupingTupleTest.scala   |   2 +-
 .../translation/CustomPartitioningTest.scala    |   2 +-
 .../JoinCustomPartitioningTest.scala            |   2 +-
 .../YARNSessionCapacitySchedulerITCase.java     |   6 +-
 .../flink/yarn/YARNSessionFIFOITCase.java       | 183 +++++++++++---
 .../org/apache/flink/yarn/YarnTestBase.java     |  78 +++++-
 .../src/main/resources/log4j-test.properties    |   2 +-
 .../org/apache/flink/yarn/FlinkYarnClient.java  |  29 ++-
 .../org/apache/flink/yarn/FlinkYarnCluster.java | 144 ++++++++---
 .../main/java/org/apache/flink/yarn/Utils.java  |   7 +-
 .../apache/flink/yarn/ApplicationClient.scala   |  20 +-
 .../flink/yarn/ApplicationMasterActor.scala     |  70 +++++-
 .../scala/org/apache/flink/yarn/Messages.scala  |  10 +
 .../java/org/apache/flink/yarn/UtilsTests.java  |  18 +-
 pom.xml                                         |   2 +-
 202 files changed, 1493 insertions(+), 1003 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 dd2a0ba..a13b322 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
@@ -42,6 +42,7 @@ import akka.pattern.Patterns;
 import akka.util.Timeout;
 
 import org.apache.commons.cli.CommandLine;
+import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.client.cli.CancelOptions;
 import org.apache.flink.client.cli.CliArgsException;
 import org.apache.flink.client.cli.CliFrontendParser;
@@ -66,7 +67,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
@@ -265,12 +266,32 @@ public class CliFrontend {
 		}
 
 		try {
-			Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName());
-
-			int parallelism = options.getParallelism();
-			int exitCode = executeProgram(program, client, parallelism);
-
-			if (yarnCluster != null) {
+			int userParallelism = options.getParallelism();
+			LOG.debug("User parallelism is set to {}", userParallelism);
+
+			Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName(), userParallelism);
+			LOG.debug("Client slots is set to {}", client.getMaxSlots());
+			if(client.getMaxSlots() != -1 && userParallelism == -1) {
+				logAndSysout("Using the parallelism provided by the remote cluster ("+client.getMaxSlots()+"). " +
+						"To use another parallelism, set it at the ./bin/flink client.");
+				userParallelism = client.getMaxSlots();
+			}
+			int exitCode = 0;
+
+			// check if detached per job yarn cluster is used to start flink
+			if(yarnCluster != null && yarnCluster.isDetached()) {
+				logAndSysout("The Flink YARN client has been started in detached mode. In order to stop " +
+						"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
+						"yarn application -kill "+yarnCluster.getApplicationId()+"\n" +
+						"Please also note that the temporary files of the YARN session in the home directoy will not be removed.");
+				executeProgram(program, client, userParallelism, false);
+			} else {
+				// regular (blocking) execution.
+				exitCode = executeProgram(program, client, userParallelism, true);
+			}
+
+			// show YARN cluster status if its not a detached YARN cluster.
+			if (yarnCluster != null && !yarnCluster.isDetached()) {
 				List<String> msgs = yarnCluster.getNewMessages();
 				if (msgs != null && msgs.size() > 1) {
 
@@ -291,7 +312,7 @@ public class CliFrontend {
 			return handleError(t);
 		}
 		finally {
-			if (yarnCluster != null) {
+			if (yarnCluster != null && !yarnCluster.isDetached()) {
 				logAndSysout("Shutting down YARN cluster");
 				yarnCluster.shutdown();
 			}
@@ -346,7 +367,7 @@ public class CliFrontend {
 			int parallelism = options.getParallelism();
 
 			LOG.info("Creating program plan dump");
-			Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName());
+			Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName(), parallelism);
 			String jsonPlan = client.getOptimizedPlanAsJson(program, parallelism);
 
 			if (jsonPlan != null) {
@@ -555,12 +576,12 @@ public class CliFrontend {
 	//  Interaction with programs and JobManager
 	// --------------------------------------------------------------------------------------------
 
-	protected int executeProgram(PackagedProgram program, Client client, int parallelism) {
-		LOG.info("Starting execution or program");
-		JobExecutionResult execResult;
+	protected int executeProgram(PackagedProgram program, Client client, int parallelism, boolean wait) {
+		LOG.info("Starting execution of program");
+		JobSubmissionResult execResult;
 		try {
 			client.setPrintStatusDuringExecution(true);
-			execResult = client.run(program, parallelism, true);
+			execResult = client.run(program, parallelism, wait);
 		}
 		catch (ProgramInvocationException e) {
 			return handleError(e);
@@ -569,15 +590,33 @@ public class CliFrontend {
 			program.deleteExtractedLibraries();
 		}
 
-		LOG.info("Program execution finished");
+		if(wait) {
+			LOG.info("Program execution finished");
+		}
 
-		// we come here after the job has finished
+		// we come here after the job has finished (or the job has been submitted)
 		if (execResult != null) {
-			System.out.println("Job Runtime: " + execResult.getNetRuntime());
-			Map<String, Object> accumulatorsResult = execResult.getAllAccumulatorResults();
-			if (accumulatorsResult.size() > 0) {
-				System.out.println("Accumulator Results: ");
-				System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult));
+			// if the job has been submitted to a detached YARN cluster, there won't be any
+			// exec results, but the object will be set (for the job id)
+			if (yarnCluster != null && yarnCluster.isDetached()) {
+				if(execResult.getJobID() == null) {
+					throw new RuntimeException("Error while starting job. No Job ID set.");
+				}
+				yarnCluster.stopAfterJob(execResult.getJobID());
+				yarnCluster.disconnect();
+				System.out.println("The Job has been submitted with JobID "+execResult.getJobID());
+				return 0;
+			}
+			if (execResult instanceof JobExecutionResult) {
+				JobExecutionResult result = (JobExecutionResult) execResult;
+				System.out.println("Job Runtime: " + result.getNetRuntime());
+				Map<String, Object> accumulatorsResult = result.getAllAccumulatorResults();
+				if (accumulatorsResult.size() > 0) {
+					System.out.println("Accumulator Results: ");
+					System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult));
+				}
+			} else {
+				LOG.info("The Job did not return an execution result");
 			}
 		}
 		return 0;
@@ -681,26 +720,47 @@ public class CliFrontend {
 		LOG.info("JobManager is at " + jmActor.path());
 		return jmActor;
 	}
-	
 
-	
-	protected Client getClient(CommandLineOptions options, ClassLoader classLoader, String programName) throws Exception {
 
+	/**
+	 *
+	 * @param options
+	 * @param classLoader
+	 * @param programName
+	 * @param userParallelism The parallelism requested by the user in the CLI frontend.
+	 * @return
+	 * @throws Exception
+	 */
+	protected Client getClient(CommandLineOptions options, ClassLoader classLoader, String programName, int userParallelism) throws Exception {
 		InetSocketAddress jobManagerAddress;
-
+		int maxSlots = -1;
 		if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) {
 			logAndSysout("YARN cluster mode detected. Switching Log4j output to console");
 
 			// user wants to run Flink in YARN cluster.
 			CommandLine commandLine = options.getCommandLine();
-			AbstractFlinkYarnClient flinkYarnClient =
-					CliFrontendParser.getFlinkYarnSessionCli().createFlinkYarnClient(commandLine);
+			AbstractFlinkYarnClient flinkYarnClient = CliFrontendParser.getFlinkYarnSessionCli().createFlinkYarnClient(commandLine);
 
 			if (flinkYarnClient == null) {
 				throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages");
 			}
+
+			// the number of slots available from YARN:
+			int yarnTmSlots = flinkYarnClient.getTaskManagerSlots();
+			if(yarnTmSlots == -1) {
+				yarnTmSlots = 1;
+			}
+			maxSlots = yarnTmSlots * flinkYarnClient.getTaskManagerCount();
+			if(userParallelism != -1) {
+				int slotsPerTM = userParallelism / flinkYarnClient.getTaskManagerCount();
+				logAndSysout("The YARN cluster has "+maxSlots+" slots available, but the user requested a parallelism of "+userParallelism+" on YARN. " +
+						"Each of the "+flinkYarnClient.getTaskManagerCount()+" TaskManagers will get "+slotsPerTM+" slots.");
+				flinkYarnClient.setTaskManagerSlots(slotsPerTM);
+			}
+
 			try {
 				yarnCluster = flinkYarnClient.deploy("Flink Application: " + programName);
+				yarnCluster.connectToCluster();
 			}
 			catch(Exception e) {
 				throw new RuntimeException("Error deploying the YARN cluster", e);
@@ -722,7 +782,7 @@ public class CliFrontend {
 						break;
 					}
 				} else {
-					logAndSysout("No status updates from YARN cluster received so far. Waiting ...");
+					logAndSysout("No status updates from the YARN cluster received so far. Waiting ...");
 				}
 
 				try {
@@ -738,7 +798,7 @@ public class CliFrontend {
 		else {
 			jobManagerAddress = getJobManagerAddress(options);
 		}
-		return new Client(jobManagerAddress, config, classLoader);
+		return new Client(jobManagerAddress, config, classLoader, maxSlots);
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 25f31e3..7352457 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
@@ -122,12 +122,6 @@ public class FlinkYarnSessionCli {
 		} else {
 			LOG.info("No path for the flink jar passed. Using the location of "+flinkYarnClient.getClass()+" to locate the jar");
 			localJarPath = new Path("file://"+flinkYarnClient.getClass().getProtectionDomain().getCodeSource().getLocation().getPath());
-			if(!localJarPath.toString().contains("uberjar")) {
-				// we need to have a proper uberjar because otherwise we don't have the required classes available on the cluster.
-				// most likely the user did try to start yarn in a regular hadoop2 flink build (not a yarn package) (using ./bin/flink -m yarn-cluster)
-				LOG.error("The detected jar file '"+localJarPath+"' is not a uberjar.");
-				return null;
-			}
 		}
 
 		flinkYarnClient.setLocalJarPath(localJarPath);
@@ -392,6 +386,10 @@ public class FlinkYarnSessionCli {
 
 			try {
 				yarnCluster = flinkYarnClient.deploy(null);
+				// only connect to cluster if its not a detached session.
+				if(!flinkYarnClient.isDetached()) {
+					yarnCluster.connectToCluster();
+				}
 			} catch (Exception e) {
 				System.err.println("Error while deploying YARN cluster: "+e.getMessage());
 				e.printStackTrace(System.err);
@@ -423,7 +421,7 @@ public class FlinkYarnSessionCli {
 
 			if (detachedMode) {
 				// print info and quit:
-				LOG.info("The Flink YARN client has been started in detached mode. In order to stop" +
+				LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
 						"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
 						"yarn application -kill "+yarnCluster.getApplicationId()+"\n" +
 						"Please also note that the temporary files of the YARN session in {} will not be removed.",

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
index 6df0f79..5ee4e5d 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
@@ -89,15 +89,20 @@ public class LocalExecutor extends PlanExecutor {
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	
+
+	public static Configuration getConfiguration(LocalExecutor le) {
+		Configuration configuration = new Configuration();
+		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, le.getTaskManagerNumSlots());
+		configuration.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, le.isDefaultOverwriteFiles());
+		return configuration;
+	}
+
 	public void start() throws Exception {
 		synchronized (this.lock) {
 			if (this.flink == null) {
 				
 				// create the embedded runtime
-				Configuration configuration = new Configuration();
-				configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, getTaskManagerNumSlots());
-				configuration.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, isDefaultOverwriteFiles());
+				Configuration configuration = getConfiguration(this);
 				// start it up
 				this.flink = new LocalFlinkMiniCluster(configuration, true);
 			} else {
@@ -158,7 +163,7 @@ public class LocalExecutor extends PlanExecutor {
 			}
 
 			try {
-				Optimizer pc = new Optimizer(new DataStatistics());
+				Optimizer pc = new Optimizer(new DataStatistics(), this.flink.getConfiguration());
 				OptimizedPlan op = pc.compile(plan);
 				
 				JobGraphGenerator jgg = new JobGraphGenerator();
@@ -186,7 +191,7 @@ public class LocalExecutor extends PlanExecutor {
 	 * @throws Exception
 	 */
 	public String getOptimizerPlanAsJSON(Plan plan) throws Exception {
-		Optimizer pc = new Optimizer(new DataStatistics());
+		Optimizer pc = new Optimizer(new DataStatistics(), getConfiguration(this));
 		OptimizedPlan op = pc.compile(plan);
 		PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
 	
@@ -242,7 +247,7 @@ public class LocalExecutor extends PlanExecutor {
 		LocalExecutor exec = new LocalExecutor();
 		try {
 			exec.start();
-			Optimizer pc = new Optimizer(new DataStatistics());
+			Optimizer pc = new Optimizer(new DataStatistics(), exec.flink.getConfiguration());
 			OptimizedPlan op = pc.compile(plan);
 			PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
index 74a7ddb..1759b65 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.PlanExecutor;
 import org.apache.flink.client.program.Client;
@@ -35,11 +36,13 @@ import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.configuration.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class RemoteExecutor extends PlanExecutor {
+	private static final Logger LOG = LoggerFactory.getLogger(RemoteExecutor.class);
 
 	private final List<String> jarFiles;
-	
 	private final InetSocketAddress address;
 	
 	public RemoteExecutor(String hostname, int port) {
@@ -86,22 +89,34 @@ public class RemoteExecutor extends PlanExecutor {
 	}
 	
 	public JobExecutionResult executePlanWithJars(JobWithJars p) throws Exception {
-		Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader());
-		return c.run(p, -1, true);
+		Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader(), -1);
+		JobSubmissionResult result = c.run(p, -1, true);
+		if(result instanceof JobExecutionResult) {
+			return (JobExecutionResult) result;
+		} else {
+			LOG.warn("The Client didn't return a JobExecutionResult");
+			return new JobExecutionResult(result.getJobID(), -1, null);
+		}
 	}
 
 	public JobExecutionResult executeJar(String jarPath, String assemblerClass, String... args) throws Exception {
 		File jarFile = new File(jarPath);
 		PackagedProgram program = new PackagedProgram(jarFile, assemblerClass, args);
 		
-		Client c = new Client(this.address, new Configuration(), program.getUserCodeClassLoader());
-		return c.run(program.getPlanWithJars(), -1, true);
+		Client c = new Client(this.address, new Configuration(), program.getUserCodeClassLoader(), -1);
+		JobSubmissionResult result = c.run(program.getPlanWithJars(), -1, true);
+		if(result instanceof JobExecutionResult) {
+			return (JobExecutionResult) result;
+		} else {
+			LOG.warn("The Client didn't return a JobExecutionResult");
+			return new JobExecutionResult(result.getJobID(), -1, null);
+		}
 	}
 
 	@Override
 	public String getOptimizerPlanAsJSON(Plan plan) throws Exception {
 		JobWithJars p = new JobWithJars(plan, this.jarFiles);
-		Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader());
+		Client c = new Client(this.address, new Configuration(), p.getUserCodeClassLoader(), -1);
 		
 		OptimizedPlan op = (OptimizedPlan) c.getOptimizedPlan(p, -1);
 		PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator();

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 9a578bc..6dff9e2 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
@@ -25,6 +25,8 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.List;
 
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.Plan;
@@ -69,6 +71,19 @@ public class Client {
 	private final Optimizer compiler;		// the compiler to compile the jobs
 	
 	private boolean printStatusDuringExecution = false;
+
+	/**
+	 * If != -1, this field specifies the total number of available slots on the cluster
+	 * conntected to the client.
+	 */
+	private int maxSlots = -1;
+
+	/**
+	 * ID of the last job submitted with this client.
+	 */
+	private JobID lastJobId = null;
+
+	private ClassLoader userCodeClassLoader; // TODO: use userCodeClassloader to deserialize accumulator results.
 	
 	// ------------------------------------------------------------------------
 	//                            Construction
@@ -80,7 +95,7 @@ public class Client {
 	 * 
 	 * @param jobManagerAddress Address and port of the job-manager.
 	 */
-	public Client(InetSocketAddress jobManagerAddress, Configuration config, ClassLoader userCodeClassLoader) {
+	public Client(InetSocketAddress jobManagerAddress, Configuration config, ClassLoader userCodeClassLoader, int maxSlots) {
 		Preconditions.checkNotNull(config, "Configuration is null");
 		this.configuration = config;
 		
@@ -88,7 +103,9 @@ public class Client {
 		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerAddress.getAddress().getHostAddress());
 		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerAddress.getPort());
 		
-		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator());
+		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration);
+		this.userCodeClassLoader = userCodeClassLoader;
+		this.maxSlots = maxSlots;
 	}
 
 	/**
@@ -112,7 +129,8 @@ public class Client {
 			throw new CompilerException("Cannot find port to job manager's RPC service in the global configuration.");
 		}
 
-		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator());
+		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration);
+		this.userCodeClassLoader = userCodeClassLoader;
 	}
 	
 	public void setPrintStatusDuringExecution(boolean print) {
@@ -126,6 +144,14 @@ public class Client {
 	public int getJobManagerPort() {
 		return this.configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
 	}
+
+	/**
+	 * @return -1 if unknown. The maximum number of available processing slots at the Flink cluster
+	 * connected to this client.
+	 */
+	public int getMaxSlots() {
+		return this.maxSlots;
+	}
 	
 	// ------------------------------------------------------------------------
 	//                      Compilation and Submission
@@ -191,8 +217,10 @@ public class Client {
 	
 	public FlinkPlan getOptimizedPlan(Plan p, int parallelism) throws CompilerException {
 		if (parallelism > 0 && p.getDefaultParallelism() <= 0) {
+			LOG.debug("Changing plan default parallelism from {} to {}",p.getDefaultParallelism(), parallelism);
 			p.setDefaultParallelism(parallelism);
 		}
+		LOG.debug("Set parallelism {}, plan default parallelism {}", parallelism, p.getDefaultParallelism());
 
 		return this.compiler.compile(p);
 	}
@@ -230,49 +258,31 @@ public class Client {
 		return job;
 	}
 
-	public JobExecutionResult run(final PackagedProgram prog, int parallelism, boolean wait) throws ProgramInvocationException {
+	public JobSubmissionResult run(final PackagedProgram prog, int parallelism, boolean wait) throws ProgramInvocationException {
 		Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader());
 		if (prog.isUsingProgramEntryPoint()) {
 			return run(prog.getPlanWithJars(), parallelism, wait);
 		}
 		else if (prog.isUsingInteractiveMode()) {
-			
-			ContextEnvironment.setAsContext(this, prog.getAllLibraries(), prog.getUserCodeClassLoader(), parallelism);
+			LOG.info("Starting program in interactive mode");
+			ContextEnvironment.setAsContext(this, prog.getAllLibraries(), prog.getUserCodeClassLoader(), parallelism, wait);
 			ContextEnvironment.enableLocalExecution(false);
-			if (wait) {
-				// invoke here
-				try {
-					prog.invokeInteractiveModeForExecution();
-				}
-				finally {
-					ContextEnvironment.enableLocalExecution(true);
-				}
+			// invoke here
+			try {
+				prog.invokeInteractiveModeForExecution();
 			}
-			else {
-				// invoke in the background
-				Thread backGroundRunner = new Thread("Program Runner") {
-					public void run() {
-						try {
-							prog.invokeInteractiveModeForExecution();
-						}
-						catch (Throwable t) {
-							LOG.error("The program execution failed.", t);
-						}
-						finally {
-							ContextEnvironment.enableLocalExecution(true);
-						}
-					}
-				};
-				backGroundRunner.start();
+			finally {
+				ContextEnvironment.enableLocalExecution(true);
 			}
-			return null;
+
+			return new JobSubmissionResult(lastJobId);
 		}
 		else {
 			throw new RuntimeException();
 		}
 	}
 	
-	public JobExecutionResult run(PackagedProgram prog, OptimizedPlan optimizedPlan, boolean wait) throws ProgramInvocationException {
+	public JobSubmissionResult run(PackagedProgram prog, OptimizedPlan optimizedPlan, boolean wait) throws ProgramInvocationException {
 		return run(optimizedPlan, prog.getAllLibraries(), wait);
 
 	}
@@ -291,17 +301,18 @@ public class Client {
 	 *                                    i.e. the job-manager is unreachable, or due to the fact that the
 	 *                                    parallel execution failed.
 	 */
-	public JobExecutionResult run(JobWithJars prog, int parallelism, boolean wait) throws CompilerException, ProgramInvocationException {
+	public JobSubmissionResult run(JobWithJars prog, int parallelism, boolean wait) throws CompilerException, ProgramInvocationException {
 		return run((OptimizedPlan) getOptimizedPlan(prog, parallelism), prog.getJarFiles(), wait);
 	}
 	
 
-	public JobExecutionResult run(OptimizedPlan compiledPlan, List<File> libraries, boolean wait) throws ProgramInvocationException {
+	public JobSubmissionResult run(OptimizedPlan compiledPlan, List<File> libraries, boolean wait) throws ProgramInvocationException {
 		JobGraph job = getJobGraph(compiledPlan, libraries);
+		this.lastJobId = job.getJobID();
 		return run(job, wait);
 	}
 
-	public JobExecutionResult run(JobGraph jobGraph, boolean wait) throws ProgramInvocationException {
+	public JobSubmissionResult run(JobGraph jobGraph, boolean wait) throws ProgramInvocationException {
 
 		final String hostname = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
 		if (hostname == null) {
@@ -335,6 +346,8 @@ public class Client {
 			}
 			else {
 				JobClient.submitJobDetached(jobGraph, client, timeout);
+				// return a "Fake" execution result with the JobId
+				return new JobSubmissionResult(jobGraph.getJobID());
 			}
 		}
 		catch (JobExecutionException e) {
@@ -347,8 +360,6 @@ public class Client {
 			actorSystem.shutdown();
 			actorSystem.awaitTermination();
 		}
-
-		return new JobExecutionResult(-1, null);
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 8d5fe17..55b579a 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
@@ -22,37 +22,51 @@ import java.io.File;
 import java.util.List;
 
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.ExecutionEnvironmentFactory;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- *
+ * Execution Environment for remote execution with the Client.
  */
 public class ContextEnvironment extends ExecutionEnvironment {
-	
+
+	private static final Logger LOG = LoggerFactory.getLogger(ContextEnvironment.class);
+
 	private final Client client;
 	
 	private final List<File> jarFilesToAttach;
 	
 	private final ClassLoader userCodeClassLoader;
+
+	private final boolean wait;
 	
 	
 	
-	public ContextEnvironment(Client remoteConnection, List<File> jarFiles, ClassLoader userCodeClassLoader) {
+	public ContextEnvironment(Client remoteConnection, List<File> jarFiles, ClassLoader userCodeClassLoader, boolean wait) {
 		this.client = remoteConnection;
 		this.jarFilesToAttach = jarFiles;
 		this.userCodeClassLoader = userCodeClassLoader;
+		this.wait = wait;
 	}
 
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
 		Plan p = createProgramPlan(jobName);
 		JobWithJars toRun = new JobWithJars(p, this.jarFilesToAttach, this.userCodeClassLoader);
-		
-		return this.client.run(toRun, getParallelism(), true);
+
+		JobSubmissionResult result = this.client.run(toRun, getParallelism(), wait);
+		if(result instanceof JobExecutionResult) {
+			return (JobExecutionResult) result;
+		} else {
+			LOG.warn("The Client didn't return a JobExecutionResult");
+			return new JobExecutionResult(result.getJobID(), -1, null);
+		}
 	}
 
 	@Override
@@ -60,7 +74,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 		Plan p = createProgramPlan("unnamed job");
 		
 		OptimizedPlan op = (OptimizedPlan) this.client.getOptimizedPlan(p, getParallelism());
-		
+
 		PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
 		return gen.getOptimizerPlanAsJSON(op);
 	}
@@ -83,15 +97,15 @@ public class ContextEnvironment extends ExecutionEnvironment {
 	// --------------------------------------------------------------------------------------------
 	
 	static void setAsContext(Client client, List<File> jarFilesToAttach, 
-				ClassLoader userCodeClassLoader, int defaultParallelism)
+				ClassLoader userCodeClassLoader, int defaultParallelism, boolean wait)
 	{
-		initializeContextEnvironment(new ContextEnvironmentFactory(client, jarFilesToAttach, userCodeClassLoader, defaultParallelism));
+		initializeContextEnvironment(new ContextEnvironmentFactory(client, jarFilesToAttach, userCodeClassLoader, defaultParallelism, wait));
 	}
 	
 	protected static void enableLocalExecution(boolean enabled) {
 		ExecutionEnvironment.enableLocalExecution(enabled);
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 	
 	public static class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
@@ -103,20 +117,23 @@ public class ContextEnvironment extends ExecutionEnvironment {
 		private final ClassLoader userCodeClassLoader;
 		
 		private final int defaultParallelism;
+
+		private final boolean wait;
 		
 
 		public ContextEnvironmentFactory(Client client, List<File> jarFilesToAttach, 
-				ClassLoader userCodeClassLoader, int defaultParallelism)
+				ClassLoader userCodeClassLoader, int defaultParallelism, boolean wait)
 		{
 			this.client = client;
 			this.jarFilesToAttach = jarFilesToAttach;
 			this.userCodeClassLoader = userCodeClassLoader;
 			this.defaultParallelism = defaultParallelism;
+			this.wait = wait;
 		}
 		
 		@Override
 		public ExecutionEnvironment createExecutionEnvironment() {
-			ContextEnvironment env = new ContextEnvironment(client, jarFilesToAttach, userCodeClassLoader);
+			ContextEnvironment env = new ContextEnvironment(client, jarFilesToAttach, userCodeClassLoader, wait);
 			if (defaultParallelism > 0) {
 				env.setParallelism(defaultParallelism);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
index 6d54b58..cb2585d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java
@@ -105,7 +105,7 @@ public class CliFrontendInfoTest {
 		}
 
 		@Override
-		protected Client getClient(CommandLineOptions options, ClassLoader loader, String programName)
+		protected Client getClient(CommandLineOptions options, ClassLoader loader, String programName, int par)
 				throws Exception
 		{
 			return new TestClient(expectedDop);
@@ -118,7 +118,7 @@ public class CliFrontendInfoTest {
 		
 		private TestClient(int expectedDop) throws Exception {
 			super(new InetSocketAddress(InetAddress.getLocalHost(), 6176),
-					new Configuration(), CliFrontendInfoTest.class.getClassLoader());
+					new Configuration(), CliFrontendInfoTest.class.getClassLoader(), -1);
 			
 			this.expectedDop = expectedDop;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
index 2712c19..3224e0f 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
@@ -22,7 +22,7 @@ import akka.actor.*;
 import akka.testkit.JavaTestKit;
 
 import org.apache.flink.client.cli.CommandLineOptions;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
index 8ed99f9..034ee4e 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
@@ -93,7 +93,7 @@ public class CliFrontendRunTest {
 		}
 
 		@Override
-		protected int executeProgram(PackagedProgram program, Client client, int parallelism) {
+		protected int executeProgram(PackagedProgram program, Client client, int parallelism, boolean wait) {
 			assertEquals(this.expectedParallelim, parallelism);
 			return 0;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
index 09a3e1a..be6c19a 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
@@ -91,7 +91,7 @@ public class ClientConnectionTest {
 			vertex.setInvokableClass(TestInvokable.class);
 
 			final JobGraph jg = new JobGraph("Test Job", vertex);
-			final Client client = new Client(unreachableEndpoint, config, getClass().getClassLoader());
+			final Client client = new Client(unreachableEndpoint, config, getClass().getClassLoader(), -1);
 
 			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
index 9278c7a..22865ed 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
@@ -24,6 +24,7 @@ import akka.actor.Status;
 import akka.actor.UntypedActor;
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.optimizer.DataStatistics;
@@ -35,7 +36,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.net.NetUtils;
 import org.junit.After;
@@ -101,7 +102,7 @@ public class ClientTest {
 		when(program.getPlanWithJars()).thenReturn(planWithJarsMock);
 		when(planWithJarsMock.getPlan()).thenReturn(planMock);
 
-		whenNew(Optimizer.class).withArguments(any(DataStatistics.class), any(CostEstimator.class)).thenReturn(this.compilerMock);
+		whenNew(Optimizer.class).withArguments(any(DataStatistics.class), any(CostEstimator.class), any(Configuration.class)).thenReturn(this.compilerMock);
 		when(compilerMock.compile(planMock)).thenReturn(optimizedPlanMock);
 
 		whenNew(JobGraphGenerator.class).withNoArguments().thenReturn(generatorMock);
@@ -139,11 +140,9 @@ public class ClientTest {
 			jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME());
 
 			Client out = new Client(config, getClass().getClassLoader());
-			JobExecutionResult result = out.run(program.getPlanWithJars(), -1, false);
+			JobSubmissionResult result = out.run(program.getPlanWithJars(), -1, false);
 
 			assertNotNull(result);
-			assertEquals(-1, result.getNetRuntime());
-			assertNull(result.getAllAccumulatorResults());
 
 			program.deleteExtractedLibraries();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java
index e2b7935..f156f77 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java
@@ -71,7 +71,8 @@ public class ExecutionPlanAfterExecutionTest implements java.io.Serializable {
 			env.getExecutionPlan();
 			env.createProgramPlan();
 		} catch (Exception e) {
-			fail("Cannot run both #getExecutionPlan and #execute.");
+			e.printStackTrace();
+			fail("Cannot run both #getExecutionPlan and #execute. Message: "+e.getMessage());
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
index 8aecff3..67b406d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
@@ -44,7 +44,7 @@ public class ExecutionPlanCreationTest {
 			InetAddress mockAddress = InetAddress.getLocalHost();
 			InetSocketAddress mockJmAddress = new InetSocketAddress(mockAddress, 12345);
 			
-			Client client = new Client(mockJmAddress, new Configuration(), getClass().getClassLoader());
+			Client client = new Client(mockJmAddress, new Configuration(), getClass().getClassLoader(), -1);
 			OptimizedPlan op = (OptimizedPlan) client.getOptimizedPlan(prg, -1);
 			assertNotNull(op);
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
index 406cfe9..68506ae 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
@@ -24,7 +24,7 @@ import java.util.Map;
  * The result of a job execution. Gives access to the execution time of the job,
  * and to all accumulators created by this job.
  */
-public class JobExecutionResult {
+public class JobExecutionResult extends JobSubmissionResult {
 
 	private long netRuntime;
 	private Map<String, Object> accumulatorResults;
@@ -32,10 +32,12 @@ public class JobExecutionResult {
 	/**
 	 * Creates a new JobExecutionResult.
 	 *
+	 * @param jobID
 	 * @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer)
 	 * @param accumulators A map of all accumulators produced by the job.
 	 */
-	public JobExecutionResult(long netRuntime, Map<String, Object> accumulators) {
+	public JobExecutionResult(JobID jobID, long netRuntime, Map<String, Object> accumulators) {
+		super(jobID);
 		this.netRuntime = netRuntime;
 		this.accumulatorResults = accumulators;
 	}
@@ -92,5 +94,6 @@ public class JobExecutionResult {
 		return (Integer) result;
 	}
 
+
 	// TODO Create convenience methods for the other shipped accumulator types
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-core/src/main/java/org/apache/flink/api/common/JobID.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobID.java b/flink-core/src/main/java/org/apache/flink/api/common/JobID.java
new file mode 100644
index 0000000..7478da4
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/JobID.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.common;
+
+import javax.xml.bind.DatatypeConverter;
+import org.apache.flink.util.AbstractID;
+import java.nio.ByteBuffer;
+
+/**
+ * Unique Job Identifier
+ */
+public final class JobID extends AbstractID {
+
+	private static final long serialVersionUID = 1L;
+	
+	public JobID() {
+		super();
+	}
+
+	public JobID(long lowerPart, long upperPart) {
+		super(lowerPart, upperPart);
+	}
+
+	public JobID(byte[] bytes) {
+		super(bytes);
+	}
+
+	public static JobID generate() {
+		return new JobID();
+	}
+
+	public static JobID fromByteArray(byte[] bytes) {
+		return new JobID(bytes);
+	}
+
+	public static JobID fromByteBuffer(ByteBuffer buf) {
+		long lower = buf.getLong();
+		long upper = buf.getLong();
+		return new JobID(lower, upper);
+	}
+
+	public static JobID fromHexString(String hexString) {
+		return new JobID(DatatypeConverter.parseHexBinary(hexString));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
new file mode 100644
index 0000000..5cea9d5
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.common;
+
+/**
+ * The result of a job submission.
+ * Contains the JobID
+ */
+public class JobSubmissionResult {
+	private JobID jobID;
+
+	public JobSubmissionResult(JobID jobID) {
+		this.jobID = jobID;
+	}
+
+	/**
+	 * Returns the JobID assigned to the job by the Flink runtime.
+	 *
+	 * @return jobID, or null if the job has been executed on a runtime without JobIDs or if the execution failed.
+	 */
+	public JobID getJobID() {
+		return jobID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java
index 78ad930..f605113 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java
@@ -100,7 +100,7 @@ public class CollectionExecutor {
 		
 		long endTime = System.currentTimeMillis();
 		Map<String, Object> accumulatorResults = AccumulatorHelper.toResultMap(accumulators);
-		return new JobExecutionResult(endTime - startTime, accumulatorResults);
+		return new JobExecutionResult(null, endTime - startTime, accumulatorResults);
 	}
 	
 	private List<?> execute(Operator<?> operator) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
index 34ebbff..9c56e61 100644
--- a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
+++ b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
@@ -29,6 +29,7 @@ log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m
 # Log output from org.apache.flink.yarn to the console. This is used by the
 # CliFrontend class when using a per-job YARN cluster.
 log4j.logger.org.apache.flink.yarn=INFO, console
+log4j.logger.org.apache.flink.client.FlinkYarnSessionCli=INFO, console
 log4j.logger.org.apache.hadoop=INFO, console
 
 log4j.appender.console=org.apache.log4j.ConsoleAppender

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
index 07aa2ce..bfd5e85 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
@@ -54,7 +54,6 @@ public class WordCount {
 	
 	public static void main(String[] args) throws Exception {
 
-
 		if(!parseParameters(args)) {
 			return;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/pom.xml
----------------------------------------------------------------------
diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml
index 55764e9..4d37aea 100644
--- a/flink-optimizer/pom.xml
+++ b/flink-optimizer/pom.xml
@@ -52,7 +52,7 @@ under the License.
 			<artifactId>flink-java</artifactId>
 			<version>${project.version}</version>
 		</dependency>
-		
+
 		<dependency>
 			<groupId>com.google.guava</groupId>
 			<artifactId>guava</artifactId>
@@ -60,4 +60,21 @@ under the License.
 		</dependency>
 	</dependencies>
 
+	<!-- Because flink-tests needs the CompilerTestBsae -->
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 c80cfc2..c4d70f6 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
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.optimizer.traversals.BinaryUnionReplacer;
 import org.apache.flink.optimizer.traversals.BranchesVisitor;
 import org.apache.flink.optimizer.traversals.GraphCreatingVisitor;
@@ -45,7 +46,6 @@ import org.apache.flink.optimizer.plan.SinkJoinerPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.postpass.OptimizerPostPass;
 import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.util.InstantiationUtil;
 
 /**
@@ -303,8 +303,8 @@ public class Optimizer {
 	 * unknown sizes and hence use only the heuristic cost functions, which result in the selection
 	 * of the most robust execution strategies.
 	 */
-	public Optimizer() {
-		this(null, new DefaultCostEstimator());
+	public Optimizer(Configuration config) {
+		this(null, new DefaultCostEstimator(), config);
 	}
 
 	/**
@@ -314,8 +314,8 @@ public class Optimizer {
 	 * @param stats
 	 *        The statistics to be used to determine the input properties.
 	 */
-	public Optimizer(DataStatistics stats) {
-		this(stats, new DefaultCostEstimator());
+	public Optimizer(DataStatistics stats, Configuration config) {
+		this(stats, new DefaultCostEstimator(), config);
 	}
 
 	/**
@@ -328,8 +328,8 @@ public class Optimizer {
 	 * 
 	 * @param estimator The cost estimator to use to cost the individual operations.
 	 */
-	public Optimizer(CostEstimator estimator) {
-		this(null, estimator);
+	public Optimizer(CostEstimator estimator, Configuration config) {
+		this(null, estimator, config);
 	}
 
 	/**
@@ -343,17 +343,17 @@ public class Optimizer {
 	 * @param estimator
 	 *        The <tt>CostEstimator</tt> to use to cost the individual operations.
 	 */
-	public Optimizer(DataStatistics stats, CostEstimator estimator) {
+	public Optimizer(DataStatistics stats, CostEstimator estimator, Configuration config) {
 		this.statistics = stats;
 		this.costEstimator = estimator;
 
 		// determine the default parallelism
 		// check for old key string first, then for new one
-		this.defaultParallelism = GlobalConfiguration.getInteger(
+		this.defaultParallelism = config.getInteger(
 				ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
 				ConfigConstants.DEFAULT_PARALLELISM);
 		// now check for new one which overwrites old values
-		this.defaultParallelism = GlobalConfiguration.getInteger(
+		this.defaultParallelism = config.getInteger(
 				ConfigConstants.DEFAULT_PARALLELISM_KEY,
 				this.defaultParallelism);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
index 1e4bafb..0c50536 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
@@ -34,6 +34,7 @@ import org.apache.flink.optimizer.util.DummyCrossStub;
 import org.apache.flink.optimizer.util.DummyInputFormat;
 import org.apache.flink.optimizer.util.DummyOutputFormat;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 2df08a0..94ff41a 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
@@ -30,6 +30,7 @@ import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction;
 import org.apache.flink.optimizer.testfunctions.SelectOneReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Assert;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java
index c7ad2da..57c53ff 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java
@@ -22,6 +22,7 @@ package org.apache.flink.optimizer;
 import static org.junit.Assert.*;
 
 import org.apache.flink.api.common.Plan;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 47efeb1..1a4cd18 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
@@ -22,6 +22,7 @@ package org.apache.flink.optimizer;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.operators.GenericDataSourceBase;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java
index eba07f1..61d407a 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.optimizer;
 
 import org.apache.flink.api.common.functions.RichCoGroupFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Assert;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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
deleted file mode 100644
index 4eed236..0000000
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CompilerTestBase.java
+++ /dev/null
@@ -1,229 +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 java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
-import org.apache.flink.api.common.operators.GenericDataSourceBase;
-import org.apache.flink.api.common.operators.Operator;
-import org.apache.flink.api.common.operators.base.BulkIterationBase;
-import org.apache.flink.optimizer.costs.DefaultCostEstimator;
-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.util.OperatingSystem;
-import org.apache.flink.util.Visitor;
-import org.junit.Before;
-
-/**
- * Base class for Optimizer tests. Offers utility methods to trigger optimization
- * of a program and to fetch the nodes in an optimizer plan that correspond
- * the the node in the program plan.
- */
-public abstract class CompilerTestBase implements java.io.Serializable {
-	
-	private static final long serialVersionUID = 1L;
-
-	protected static final String IN_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/random";
-	
-	protected static final String OUT_FILE = OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/null";
-	
-	protected static final int DEFAULT_PARALLELISM = 8;
-	
-	private static final String CACHE_KEY = "cachekey";
-	
-	// ------------------------------------------------------------------------
-	
-	protected transient DataStatistics dataStats;
-	
-	protected transient Optimizer withStatsCompiler;
-	
-	protected transient Optimizer noStatsCompiler;
-	
-	private transient int statCounter;
-	
-	// ------------------------------------------------------------------------	
-	
-	@Before
-	public void setup() {
-		this.dataStats = new DataStatistics();
-		this.withStatsCompiler = new Optimizer(this.dataStats, new DefaultCostEstimator());
-		this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
-		
-		this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator());
-		this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM);
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	public OptimizedPlan compileWithStats(Plan p) {
-		return this.withStatsCompiler.compile(p);
-	}
-	
-	public OptimizedPlan compileNoStats(Plan p) {
-		return this.noStatsCompiler.compile(p);
-	}
-	
-	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, long size, float recordWidth) {
-		setSourceStatistics(source, new FileBaseStatistics(Long.MAX_VALUE, size, recordWidth));
-	}
-	
-	public void setSourceStatistics(GenericDataSourceBase<?, ?> source, FileBaseStatistics stats) {
-		final String key = CACHE_KEY + this.statCounter++;
-		this.dataStats.cacheBaseStatistics(stats, key);
-		source.setStatisticsKey(key);
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) {
-		return new OptimizerPlanNodeResolver(plan);
-	}
-	
-	public static final class OptimizerPlanNodeResolver {
-		
-		private final Map<String, ArrayList<PlanNode>> map;
-		
-		OptimizerPlanNodeResolver(OptimizedPlan p) {
-			HashMap<String, ArrayList<PlanNode>> map = new HashMap<String, ArrayList<PlanNode>>();
-			
-			for (PlanNode n : p.getAllNodes()) {
-				Operator<?> c = n.getOriginalOptimizerNode().getOperator();
-				String name = c.getName();
-				
-				ArrayList<PlanNode> list = map.get(name);
-				if (list == null) {
-					list = new ArrayList<PlanNode>(2);
-					map.put(name, list);
-				}
-				
-				// check whether this node is a child of a node with the same contract (aka combiner)
-				boolean shouldAdd = true;
-				for (Iterator<PlanNode> iter = list.iterator(); iter.hasNext();) {
-					PlanNode in = iter.next();
-					if (in.getOriginalOptimizerNode().getOperator() == c) {
-						// is this the child or is our node the child
-						if (in instanceof SingleInputPlanNode && n instanceof SingleInputPlanNode) {
-							SingleInputPlanNode thisNode = (SingleInputPlanNode) n;
-							SingleInputPlanNode otherNode = (SingleInputPlanNode) in;
-							
-							if (thisNode.getPredecessor() == otherNode) {
-								// other node is child, remove it
-								iter.remove();
-							} else if (otherNode.getPredecessor() == thisNode) {
-								shouldAdd = false;
-							}
-						} else {
-							throw new RuntimeException("Unrecodnized case in test.");
-						}
-					}
-				}
-				
-				if (shouldAdd) {
-					list.add(n);
-				}
-			}
-			
-			this.map = map;
-		}
-		
-		
-		@SuppressWarnings("unchecked")
-		public <T extends PlanNode> T getNode(String name) {
-			List<PlanNode> nodes = this.map.get(name);
-			if (nodes == null || nodes.isEmpty()) {
-				throw new RuntimeException("No node found with the given name.");
-			} else if (nodes.size() != 1) {
-				throw new RuntimeException("Multiple nodes found with the given name.");
-			} else {
-				return (T) nodes.get(0);
-			}
-		}
-		
-		@SuppressWarnings("unchecked")
-		public <T extends PlanNode> T getNode(String name, Class<? extends Function> stubClass) {
-			List<PlanNode> nodes = this.map.get(name);
-			if (nodes == null || nodes.isEmpty()) {
-				throw new RuntimeException("No node found with the given name and stub class.");
-			} else {
-				PlanNode found = null;
-				for (PlanNode node : nodes) {
-					if (node.getClass() == stubClass) {
-						if (found == null) {
-							found = node;
-						} else {
-							throw new RuntimeException("Multiple nodes found with the given name and stub class.");
-						}
-					}
-				}
-				if (found == null) {
-					throw new RuntimeException("No node found with the given name and stub class.");
-				} else {
-					return (T) found;
-				}
-			}
-		}
-		
-		public List<PlanNode> getNodes(String name) {
-			List<PlanNode> nodes = this.map.get(name);
-			if (nodes == null || nodes.isEmpty()) {
-				throw new RuntimeException("No node found with the given name.");
-			} else {
-				return new ArrayList<PlanNode>(nodes);
-			}
-		}
-	}
-
-	/**
-	 * Collects all DataSources of a plan to add statistics
-	 *
-	 */
-	public static class SourceCollectorVisitor implements Visitor<Operator<?>> {
-		
-		protected final List<GenericDataSourceBase<?, ?>> sources = new ArrayList<GenericDataSourceBase<?, ?>>(4);
-
-		@Override
-		public boolean preVisit(Operator<?> visitable) {
-			
-			if(visitable instanceof GenericDataSourceBase) {
-				sources.add((GenericDataSourceBase<?, ?>) visitable);
-			}
-			else if(visitable instanceof BulkIterationBase) {
-				((BulkIterationBase<?>) visitable).getNextPartialSolution().accept(this);
-			}
-			
-			return true;
-		}
-
-		@Override
-		public void postVisit(Operator<?> visitable) {}
-		
-		public List<GenericDataSourceBase<?, ?>> getSources() {
-			return this.sources;
-		}
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java
index aaee975..bb3aa47 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 3b7eae7..7865861 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
@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.optimizer.plan.SourcePlanNode;
 import org.apache.flink.runtime.operators.DriverStrategy;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.java
index 77d185d..76b3b0e 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.java
@@ -39,6 +39,7 @@ import org.apache.flink.optimizer.util.DummyOutputFormat;
 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.optimizer.util.CompilerTestBase;
 import org.apache.flink.types.DoubleValue;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.types.LongValue;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java
index 6dadc19..52e9a2d 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.util.DummyInputFormat;
 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.optimizer.util.CompilerTestBase;
 import org.apache.flink.types.IntValue;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 810ec0e..0afbe93 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
@@ -21,6 +21,7 @@ package org.apache.flink.optimizer;
 import static org.junit.Assert.*;
 
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java
index e65758f..34fc085 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction;
 import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor;
 import org.apache.flink.optimizer.testfunctions.IdentityMapper;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings({"serial", "unchecked"})

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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
index a54136a..8236f10 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java
@@ -29,6 +29,7 @@ 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.optimizer.util.CompilerTestBase;
 import org.junit.Assert;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.record.operators.FileDataSink;

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java
index 2b42f85..72effc1 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java
@@ -29,6 +29,7 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
index 16684dc..f42eb02 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
@@ -32,6 +32,7 @@ import org.apache.flink.optimizer.dataproperties.PartitioningProperty;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b0d4076/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 31f71d1..84f6377 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
@@ -22,6 +22,7 @@ import static org.junit.Assert.*;
 
 import org.apache.flink.optimizer.testfunctions.IdentityMapper;
 import org.apache.flink.optimizer.testfunctions.SelectOneReducer;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.junit.Test;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.DataSet;